Skip to content

Commit

Permalink
Merge #109365
Browse files Browse the repository at this point in the history
109365: pkg/util/log: send sentry report when logging already active r=Santamaura a=abarganier

If a log config is used to write a log msg within the logging package, that log config is marked as "active". After this point, if another caller attempts to apply a new log config, the code previously would panic.

This case rarely appears - we've only had one customer bring it up (see: #84638). Generally, it occurs in the real world when a SIGHUP is sent to the cockroach process very early in the initialization process. At this point, a barebones log config is in place, prior to the parsing & applying of the actual log config. SIGHUP triggers a log flush, which if timed properly, sets this barebones log config to "active". This means that later in the init process, when we parse & apply the actual config, we hit this panic since a config has already been marked "active".

The panic on startup is not a great look with customers, and the scenario described above is fairly harmless. Therefore, a less abrasive approach would be to:

1. Continue panicking in the case of a non-release CRDB binary.
2. Log & report an error to Sentry in the case of a release CRDB binary.

This makes the impact of hitting this scenario less abrasive to customers, while maintaining reporting. It also allows us to keep things extra noisy during development and testing, in case actual bugs get introduced.

This patch implements this. As usual, since we're working within pkg/util/log, dependency injection had to be used.

Release note: none

Informs: #84638

Co-authored-by: Alex Barganier <abarganier@cockroachlabs.com>
  • Loading branch information
craig[bot] and abarganier committed Aug 30, 2023
2 parents f464c06 + b277506 commit 8b7fb0c
Show file tree
Hide file tree
Showing 5 changed files with 60 additions and 2 deletions.
3 changes: 2 additions & 1 deletion pkg/cli/log_flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/channel"
"github.com/cockroachdb/cockroach/pkg/util/log/logconfig"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
"github.com/cockroachdb/cockroach/pkg/util/log/severity"
"github.com/cockroachdb/errors"
"github.com/spf13/cobra"
Expand Down Expand Up @@ -52,7 +53,7 @@ func setupLogging(ctx context.Context, cmd *cobra.Command, isServerCmd, applyCon

// Sanity check to prevent misuse of API.
if active, firstUse := log.IsActive(); active {
panic(errors.Newf("logging already active; first used at:\n%s", firstUse))
logcrash.ReportOrPanic(ctx, nil /* sv */, "logging already active; first used at:\n%s", firstUse)
}

// Try to derive a default directory from the first store,
Expand Down
2 changes: 2 additions & 0 deletions pkg/util/log/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ go_library(
"metric.go",
"redact.go",
"registry.go",
"report.go",
"sinks.go",
"stderr_redirect.go",
"stderr_redirect_unix.go",
Expand All @@ -55,6 +56,7 @@ go_library(
"//pkg/base/serverident",
"//pkg/build",
"//pkg/cli/exit",
"//pkg/settings",
"//pkg/testutils/skip",
"//pkg/util",
"//pkg/util/allstacks",
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/log/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,7 +93,7 @@ func IsActive() (active bool, firstUse string) {
func ApplyConfig(config logconfig.Config) (logShutdownFn func(), err error) {
// Sanity check.
if active, firstUse := IsActive(); active {
panic(errors.Newf("logging already active; first use:\n%s", firstUse))
reportOrPanic(context.Background(), nil /* sv */, "logging already active; first use:\n%s", firstUse)
}

// Our own cancellable context to stop the secondary loggers below.
Expand Down
5 changes: 5 additions & 0 deletions pkg/util/log/logcrash/crash_reporting.go
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,11 @@ var (
globalSettings atomic.Value
)

func init() {
// Inject ReportOrPanic into the log package.
log.SetReportOrPanicFn(ReportOrPanic)
}

// SetGlobalSettings sets the *settings.Values container that will be refreshed
// at runtime -- ideally we should have no other *Values containers floating
// around, as they will be stale / lies.
Expand Down
50 changes: 50 additions & 0 deletions pkg/util/log/report.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,50 @@
// Copyright 2023 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package log

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/errors"
)

// reportOrPanic is expected to be injected by pkg/logcrash to avoid a circular
// dependency.
//
// reportOrPanic, once injected, either reports an error to sentry, if run from a
// release binary, or panics, if triggered in tests. This is intended to be used for
// failing assertions which are recoverable but serious enough to report and to
// cause tests to fail.
//
// NB: The format string should not contain any sensitive data, and unsafe reportables
// will be redacted before reporting.
//
// NB2: We provide a default definition that always panics. This is because pkg/logcrash
// injects the real definition in its `init()` function, but this function is used
// in pkg/util/log's *own* `init()` function. pkg/logcrash imports pkg/util/log, not
// the other way around, so pkg/util/log's `init()` function is called first.
// Therefore, to avoid a nil reference in the event that our own package's `init()`
// needs to use reportOrPanic, we provide this default definition.
var reportOrPanic = func(ctx context.Context, sv *settings.Values, format string, reportables ...interface{}) {
err := errors.Newf("%s", fmt.Sprintf(format, reportables))
panic(err)
}

// SetReportOrPanicFn injects a definition for reportOrPanic into pkg/util/log.
//
// Used to avoid a dependency cycle.
func SetReportOrPanicFn(
fn func(ctx context.Context, sv *settings.Values, format string, reportables ...interface{}),
) {
reportOrPanic = fn
}

0 comments on commit 8b7fb0c

Please sign in to comment.