Skip to content

Commit

Permalink
pkg/*: replace uses of debug.Stack() with debugutil.Stack()
Browse files Browse the repository at this point in the history
A previous commit introduced the debugutil.Stack() function which
wraps the debug.Stack() output in "SafeStack" type to make sure that it
does not get redacted.

This commit replaces all the current uses of debug.Stack() with
debugutil.Stack(). With one exception - there is still a call to
debug.Stack() in pkg/util/log/clog.go. Changing this is not that
straightforward. But it's ok keep it for now because this does not go
through the redaction flow.

Part of: CRDB-15292
Epic: CRDB-37533
Release note: None
  • Loading branch information
arjunmahishi committed Dec 9, 2024
1 parent 5d83c6c commit 5ff2ff2
Show file tree
Hide file tree
Showing 40 changed files with 79 additions and 63 deletions.
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,7 @@ go_library(
"//pkg/roachprod/vm",
"//pkg/testutils/release",
"//pkg/util/ctxgroup",
"//pkg/util/debugutil",
"//pkg/util/intsets",
"//pkg/util/randutil",
"//pkg/util/retry",
Expand Down
4 changes: 2 additions & 2 deletions pkg/cmd/roachtest/roachtestutil/mixedversion/runner.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ import (
"os"
"path/filepath"
"regexp"
"runtime/debug"
"strings"
"sync"
"sync/atomic"
Expand All @@ -30,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachprod/install"
"github.com/cockroachdb/cockroach/pkg/roachprod/logger"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -868,7 +868,7 @@ func panicAsError(l *logger.Logger, f func() error) (retErr error) {
// logPanicToErr logs the panic stack trace and returns an error with the
// panic message.
func logPanicToErr(l *logger.Logger, r interface{}) error {
l.Printf("panic stack trace:\n%s", string(debug.Stack()))
l.Printf("panic stack trace:\n%s", debugutil.Stack())
return fmt.Errorf("panic (stack trace above): %v", r)
}

Expand Down
1 change: 1 addition & 0 deletions pkg/config/zonepb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ go_library(
"//pkg/sql/pgwire/pgcode",
"//pkg/sql/pgwire/pgerror",
"//pkg/sql/sem/tree",
"//pkg/util/debugutil",
"//pkg/util/envutil",
"//pkg/util/log",
"@com_github_cockroachdb_errors//:errors",
Expand Down
6 changes: 3 additions & 3 deletions pkg/config/zonepb/zone_yaml.go
Original file line number Diff line number Diff line change
Expand Up @@ -7,10 +7,10 @@ package zonepb

import (
"fmt"
"runtime/debug"
"sort"
"strings"

"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/errors"
"github.com/gogo/protobuf/proto"
"gopkg.in/yaml.v2"
Expand Down Expand Up @@ -50,13 +50,13 @@ var _ yaml.Unmarshaler = &ConstraintsConjunction{}
// MarshalYAML implements yaml.Marshaler.
func (c ConstraintsConjunction) MarshalYAML() (interface{}, error) {
return nil, fmt.Errorf(
"MarshalYAML should never be called directly on Constraints (%v): %v", c, debug.Stack())
"MarshalYAML should never be called directly on Constraints (%v): %v", c, debugutil.Stack())
}

// UnmarshalYAML implements yaml.Marshaler.
func (c *ConstraintsConjunction) UnmarshalYAML(unmarshal func(interface{}) error) error {
return fmt.Errorf(
"UnmarshalYAML should never be called directly on Constraints: %v", debug.Stack())
"UnmarshalYAML should never be called directly on Constraints: %v", debugutil.Stack())
}

// ConstraintsList is an alias for a slice of Constraints that can be
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -65,6 +65,7 @@ go_library(
"//pkg/util/buildutil",
"//pkg/util/circuit",
"//pkg/util/ctxgroup",
"//pkg/util/debugutil",
"//pkg/util/envutil",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/future",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ package kvcoord
import (
"context"
"math/rand"
"runtime/debug"

"github.com/cockroachdb/cockroach/pkg/build"
"github.com/cockroachdb/cockroach/pkg/kv"
Expand All @@ -17,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/buildutil"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
Expand Down Expand Up @@ -734,7 +734,7 @@ func (tc *TxnCoordSender) maybeRejectClientLocked(
// If the client is trying to do anything other than rollback, it is
// unexpected for it to find the transaction already in a txnFinalized
// state. This may be a bug, so log a stack trace.
stack := string(debug.Stack())
stack := debugutil.Stack()
log.Errorf(ctx, "%s. stack:\n%s", msg, stack)
}
reason := kvpb.TransactionStatusError_REASON_UNKNOWN
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 @@ -207,6 +207,7 @@ go_library(
"//pkg/util/buildutil",
"//pkg/util/circuit",
"//pkg/util/ctxgroup",
"//pkg/util/debugutil",
"//pkg/util/encoding",
"//pkg/util/envutil",
"//pkg/util/errorutil",
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/concurrency/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_library(
"//pkg/storage/enginepb",
"//pkg/util/buildutil",
"//pkg/util/container/list",
"//pkg/util/debugutil",
"//pkg/util/hlc",
"//pkg/util/humanizeutil",
"//pkg/util/log",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/concurrency/concurrency_manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ package concurrency
import (
"context"
"fmt"
"runtime/debug"
"sync"

"github.com/cockroachdb/cockroach/pkg/kv"
Expand All @@ -22,6 +21,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
Expand Down Expand Up @@ -302,7 +302,7 @@ func (m *managerImpl) sequenceReqWithGuard(
panic(redact.Safe(fmt.Sprintf("must not be holding latches\n"+
"this is tracked in github.com/cockroachdb/cockroach/issues/77663; please comment if seen\n"+
"eval_kind=%d, holding_latches=%t, branch=%d, first_iteration=%t, stack=\n%s",
g.EvalKind, g.HoldingLatches(), branch, firstIteration, string(debug.Stack()))))
g.EvalKind, g.HoldingLatches(), branch, firstIteration, debugutil.Stack())))
}
log.Event(ctx, "optimistic failed, so waiting for latches")
g.lg, err = m.lm.WaitUntilAcquired(ctx, g.lg)
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/intentresolver/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/storage/enginepb",
"//pkg/util/debugutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/metric",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/intentresolver/intent_resolver.go
Original file line number Diff line number Diff line change
Expand Up @@ -9,7 +9,6 @@ import (
"bytes"
"context"
"math"
"runtime/debug"
"sort"
"time"

Expand All @@ -24,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/quotapool"
Expand Down Expand Up @@ -1034,7 +1034,7 @@ func (ir *IntentResolver) resolveIntents(
// if !build.IsRelease() && h == (kvpb.AdmissionHeader{}) && ir.everyAdmissionHeaderMissing.ShouldLog() {
if false {
log.Warningf(ctx,
"test-only warning: if you see this, please report to https://github.com/cockroachdb/cockroach/issues/112680. empty admission header provided by %s", string(debug.Stack()))
"test-only warning: if you see this, please report to https://github.com/cockroachdb/cockroach/issues/112680. empty admission header provided by %s", debugutil.Stack())
}
// Send the requests ...
if opts.sendImmediately {
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 @@ -21,6 +21,7 @@ go_library(
"//pkg/roachpb",
"//pkg/settings",
"//pkg/settings/cluster",
"//pkg/util/debugutil",
"//pkg/util/errorutil",
"//pkg/util/hlc",
"//pkg/util/log",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/kvserverbase/syncing_write.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,12 @@ package kvserverbase
import (
"context"
"os"
"runtime/debug"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/settings/cluster"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -53,7 +53,7 @@ func LimitBulkIOWrite(ctx context.Context, limiter *rate.Limiter, cost int) erro

if d := timeutil.Since(begin); d > bulkIOWriteLimiterLongWait {
log.Warningf(ctx, "bulk io write limiter took %s (>%s):\n%s",
d, bulkIOWriteLimiterLongWait, debug.Stack())
d, bulkIOWriteLimiterLongWait, debugutil.Stack())
}
return nil
}
Expand Down
8 changes: 4 additions & 4 deletions pkg/kv/kvserver/metrics.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,7 +8,6 @@ package kvserver
import (
"context"
"fmt"
"runtime/debug"
"sync/atomic"
"time"

Expand All @@ -24,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/storage/disk"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/storage/fs"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/cockroach/pkg/util/metric/aggmetric"
Expand Down Expand Up @@ -3004,15 +3004,15 @@ type tenantMetricsRef struct {
// in assertions on failure.
_stack struct {
syncutil.Mutex
string
debugutil.SafeStack
}
}

func (ref *tenantMetricsRef) assert(ctx context.Context) {
if atomic.LoadInt32(&ref._state) != 0 {
ref._stack.Lock()
defer ref._stack.Unlock()
log.FatalfDepth(ctx, 1, "tenantMetricsRef already finalized in:\n%s", ref._stack.string)
log.FatalfDepth(ctx, 1, "tenantMetricsRef already finalized in:\n%s", ref._stack.SafeStack)
}
}

Expand Down Expand Up @@ -3162,7 +3162,7 @@ func (sm *TenantsStorageMetrics) releaseTenant(ctx context.Context, ref *tenantM
return // unreachable
}
ref._stack.Lock()
ref._stack.string = string(debug.Stack())
ref._stack.SafeStack = debugutil.Stack()
ref._stack.Unlock()
m.mu.Lock()
defer m.mu.Unlock()
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/spanset/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ go_library(
"//pkg/roachpb",
"//pkg/storage",
"//pkg/storage/fs",
"//pkg/util/debugutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/protoutil",
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/spanset/spanset.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,12 +8,12 @@ package spanset
import (
"context"
"fmt"
"runtime/debug"
"strings"
"sync"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
Expand Down Expand Up @@ -360,7 +360,7 @@ func (s *SpanSet) checkAllowed(
}
}

return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, s, debug.Stack())
return errors.Errorf("cannot %s undeclared span %s\ndeclared:\n%s\nstack:\n%s", access, span, s, debugutil.Stack())
}

// contains returns whether s1 contains s2. Unlike Span.Contains, this function
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/store_merge.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,10 +8,10 @@ package kvserver
import (
"context"
"runtime"
"runtime/debug"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/readsummary/rspb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
Expand All @@ -38,7 +38,7 @@ func (s *Store) maybeAssertNoHole(ctx context.Context, from, to roachpb.RKey) fu
}

goroutineStopped := make(chan struct{})
caller := string(debug.Stack())
caller := debugutil.Stack()
if from.Equal(roachpb.RKeyMax) {
// There will be a hole to the right of RKeyMax but it's just the end of
// the addressable keyspace.
Expand Down
1 change: 1 addition & 0 deletions pkg/spanconfig/spanconfigsqlwatcher/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/types",
"//pkg/util",
"//pkg/util/debugutil",
"//pkg/util/hlc",
"//pkg/util/log",
"//pkg/util/log/logcrash",
Expand Down
5 changes: 2 additions & 3 deletions pkg/spanconfig/spanconfigsqlwatcher/zonesdecoder.go
Original file line number Diff line number Diff line change
Expand Up @@ -6,15 +6,14 @@
package spanconfigsqlwatcher

import (
"runtime/debug"

"github.com/cockroachdb/cockroach/pkg/keys"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/systemschema"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/errors"
)

Expand All @@ -40,7 +39,7 @@ func (zd *zonesDecoder) DecodePrimaryKey(key roachpb.Key) (descpb.ID, error) {
types := []*types.T{tbl.PublicColumns()[0].GetType()}
startKeyRow := make([]rowenc.EncDatum, 1)
if _, err := rowenc.DecodeIndexKey(zd.codec, startKeyRow, nil /* colDirs */, key); err != nil {
return descpb.InvalidID, errors.NewAssertionErrorWithWrappedErrf(err, "failed to decode key in system.zones %v %v", key, string(debug.Stack()))
return descpb.InvalidID, errors.NewAssertionErrorWithWrappedErrf(err, "failed to decode key in system.zones %v %v", key, debugutil.Stack())
}
if err := startKeyRow[0].EnsureDecoded(types[0], &zd.alloc); err != nil {
return descpb.InvalidID, errors.NewAssertionErrorWithWrappedErrf(err, "failed to decode key in system.zones %v", key)
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/lease/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -144,6 +144,7 @@ go_test(
"//pkg/util/admission",
"//pkg/util/allstacks",
"//pkg/util/ctxgroup",
"//pkg/util/debugutil",
"//pkg/util/encoding",
"//pkg/util/hlc",
"//pkg/util/leaktest",
Expand Down
6 changes: 3 additions & 3 deletions pkg/sql/catalog/lease/lease_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ import (
"context"
gosql "database/sql"
"fmt"
"runtime/debug"
"strings"
"sync"
"sync/atomic"
Expand Down Expand Up @@ -64,6 +63,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/testutils/testcluster"
"github.com/cockroachdb/cockroach/pkg/util/allstacks"
"github.com/cockroachdb/cockroach/pkg/util/ctxgroup"
"github.com/cockroachdb/cockroach/pkg/util/debugutil"
"github.com/cockroachdb/cockroach/pkg/util/encoding"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
Expand Down Expand Up @@ -936,7 +936,7 @@ func TestDescriptorRefreshOnRetry(t *testing.T) {
RemoveOnceDereferenced: true,
LeaseAcquiredEvent: func(desc catalog.Descriptor, _ error) {
if desc.GetName() == "foo" {
log.Infof(ctx, "lease acquirer stack trace: %s", debug.Stack())
log.Infof(ctx, "lease acquirer stack trace: %s", debugutil.Stack())
atomic.AddInt32(&fooAcquiredCount, 1)
}
},
Expand Down Expand Up @@ -1323,7 +1323,7 @@ func TestLeaseRenewedAutomatically(testingT *testing.T) {
// see a block event dump a stack to aid in debugging.
log.Infof(ctx,
"Lease acquisition of ID %d resulted in a block event. Stack trace to follow:\n%s",
id, debug.Stack())
id, debugutil.Stack())
},
},
}
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/logictest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -89,6 +89,7 @@ go_library(
"//pkg/testutils/serverutils",
"//pkg/testutils/skip",
"//pkg/testutils/sqlutils",
"//pkg/util/debugutil",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/metamorphic",
Expand Down
Loading

0 comments on commit 5ff2ff2

Please sign in to comment.