Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
5 changes: 0 additions & 5 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,6 @@ import (
"github.com/cockroachdb/cockroach/pkg/util"
"github.com/cockroachdb/cockroach/pkg/util/cgroups"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
"github.com/cockroachdb/cockroach/pkg/util/grpcutil"
"github.com/cockroachdb/cockroach/pkg/util/humanizeutil"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/logcrash"
Expand Down Expand Up @@ -515,10 +514,6 @@ func runStartInternal(
}
stopper.SetTracer(serverCfg.BaseConfig.AmbientCtx.Tracer)

// We don't care about GRPCs fairly verbose logs in most client commands,
// but when actually starting a server, we enable them.
grpcutil.LowerSeverity(severity.WARNING)

// Tweak GOMAXPROCS if we're in a cgroup / container that has cpu limits set.
// The GO default for GOMAXPROCS is NumCPU(), however this is less
// than ideal if the cgroup is limited to a number lower than that.
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/closedts/sidetransport/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/util/hlc",
"//pkg/util/intsets",
"//pkg/util/log",
"//pkg/util/netutil",
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
Expand Down
4 changes: 3 additions & 1 deletion pkg/kv/kvserver/closedts/sidetransport/sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,9 +32,11 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/intsets"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"google.golang.org/grpc"
)

Expand Down Expand Up @@ -794,7 +796,7 @@ func (r *rpcConn) run(ctx context.Context, stopper *stop.Stopper) {
return
}
if err := r.maybeConnect(ctx, stopper); err != nil {
if everyN.ShouldLog() {
if !errors.HasType(err, (*netutil.InitialHeartbeatFailedError)(nil)) && everyN.ShouldLog() {
log.Infof(ctx, "side-transport failed to connect to n%d: %s", r.nodeID, err)
}
time.Sleep(errSleepTime)
Expand Down
4 changes: 1 addition & 3 deletions pkg/util/grpcutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -7,6 +7,7 @@ go_library(
"fast_metadata.go",
"grpc_err_redaction.go",
"grpc_log.go",
"grpc_log_legacy.go",
"grpc_util.go",
],
importpath = "github.com/cockroachdb/cockroach/pkg/util/grpcutil",
Expand All @@ -16,8 +17,6 @@ go_library(
"//pkg/util/log",
"//pkg/util/log/severity",
"//pkg/util/netutil",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"@com_github_cockroachdb_circuitbreaker//:circuitbreaker",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_errors//errbase",
Expand Down Expand Up @@ -48,7 +47,6 @@ go_test(
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/log/severity",
"//pkg/util/timeutil",
"@com_github_cockroachdb_circuitbreaker//:circuitbreaker",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
Expand Down
215 changes: 59 additions & 156 deletions pkg/util/grpcutil/grpc_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,21 +12,25 @@ package grpcutil

import (
"context"
"fmt"
"math"
"os"
"regexp"
"strconv"
"strings"
"time"

"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/severity"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"google.golang.org/grpc/grpclog"
)

func init() {
// gRPC's logs are not very useful outside of debugging situations.
// vmodule works for them, with special semantics as follows:
// 1: log error and fatal only;
// 2: log warning, error, fatal;
// 3: log everything.
//
// For example, `--vmodule=clientconn=2` logs everything except INFO.
LowerSeverity(severity.ERROR)
}

Expand Down Expand Up @@ -95,189 +99,88 @@ func getGRPCLogger(inputSeverity log.Severity, envSev, envVer string) *grpcLogge
// Specifically, we get a stack frame that appears as "<autogenerated>", which
// is not useful in logs.
var _ grpclog.LoggerV2 = (*grpcLogger)(nil)

// We use a depth of 2 throughout this file because all logging
// calls originate from the logging adapter file in grpc, which is
// an additional stack frame away from the actual logging site.
const depth2 = 2
var _ grpclog.DepthLoggerV2 = (*grpcLogger)(nil)

type grpcLogger struct {
sev log.Severity
grpcVerbosityLevel int
}

func (l *grpcLogger) vDepth(i int, depth int) bool {
if i < 0 {
i = 0
}
if i > math.MaxInt32 {
i = math.MaxInt32
}

// If GRPC_GO_LOG_VERBOSITY_LEVEL is less restrictive, it prevails.
if i <= l.grpcVerbosityLevel {
return true
}
// Otherwise, our logger decides.
return log.VDepth(log.Level(i) /* level */, depth+1)
}

func (l *grpcLogger) shouldLog(incomingSeverity log.Severity, depth int) bool {
// If the incoming severity is at or above threshold, log.
if l.sev <= incomingSeverity {
return true
}
// If verbose logging is on at all (either for our
// logger or via the grpc env var), log all severities.
return l.vDepth(1, depth+1)
}

func (l *grpcLogger) Info(args ...interface{}) {
if !l.shouldLog(severity.INFO, depth2) {
return
}
log.InfofDepth(context.TODO(), depth2, "", args...)
}
// gRPC doesn't consistently propagate all of its stack frame increments. At
// time of writing it's missing the last one. Easy to see by setting a
// breakpoint on WarningDepth and looking at a stack from TestHeartbeatDialback.
// We add once more to account for the stack frame we're introducing.
const depthLoggerDelta = 2

func (l *grpcLogger) Infoln(args ...interface{}) {
if !l.shouldLog(severity.INFO, depth2) {
return
func (l *grpcLogger) sanitize(args []interface{}) []interface{} {
// NB: redaction would be nice, but the gRPC log args are actually just
// a subsystem like "[core]" followed by a free-form message that often
// has a multi-line JSON object in it (but not in any structured way),
// so there's really no way to work with it. Best we can do is throw
// away the many newlines.
sl := make([]interface{}, 0, len(args))
for _, arg := range args {
sl = append(sl, strings.Replace(fmt.Sprint(arg), "\n", " ", -1))
}
log.InfofDepth(context.TODO(), depth2, "", args...)
return sl
}

func (l *grpcLogger) Infof(format string, args ...interface{}) {
if !l.shouldLog(severity.INFO, depth2) {
func (l *grpcLogger) InfoDepth(depth int, args ...interface{}) {
depth += depthLoggerDelta
if !l.shouldLog(severity.INFO, depth) {
return
}
log.InfofDepth(context.TODO(), depth2, format, args...)
log.InfofDepth(context.TODO(), depth, "", l.sanitize(args)...)
}

func (l *grpcLogger) Warning(args ...interface{}) {
if !l.shouldLog(severity.WARNING, depth2) {
return
}
if !l.shouldPrintWarning(depth2, args...) {
func (l *grpcLogger) WarningDepth(depth int, args ...interface{}) {
depth += depthLoggerDelta
if !l.shouldLog(severity.WARNING, depth) {
return
}
log.WarningfDepth(context.TODO(), depth2, "", args...)
log.WarningfDepth(context.TODO(), depth, "", l.sanitize(args)...)
}

func (l *grpcLogger) Warningln(args ...interface{}) {
if !l.shouldLog(severity.WARNING, depth2) {
func (l *grpcLogger) ErrorDepth(depth int, args ...interface{}) {
depth += depthLoggerDelta
if !l.shouldLog(severity.ERROR, depth) {
return
}
log.WarningfDepth(context.TODO(), depth2, "", args...)
log.ErrorfDepth(context.TODO(), depth, "", l.sanitize(args)...)
}

func (l *grpcLogger) Warningf(format string, args ...interface{}) {
if !l.shouldLog(severity.WARNING, depth2) {
return
}
log.WarningfDepth(context.TODO(), depth2, format, args...)
func (l *grpcLogger) FatalDepth(depth int, args ...interface{}) {
depth += depthLoggerDelta
// Never suppress fatals.
log.FatalfDepth(context.TODO(), depth, "", l.sanitize(args)...)
}

func (l *grpcLogger) Error(args ...interface{}) {
if !l.shouldLog(severity.ERROR, depth2) {
return
func (l *grpcLogger) vDepth(i int, depth int) bool {
if i < 0 {
i = 0
}
log.ErrorfDepth(context.TODO(), depth2, "", args...)
}

func (l *grpcLogger) Errorln(args ...interface{}) {
if !l.shouldLog(severity.ERROR, depth2) {
return
if i > math.MaxInt32 {
i = math.MaxInt32
}
log.ErrorfDepth(context.TODO(), depth2, "", args...)
}

func (l *grpcLogger) Errorf(format string, args ...interface{}) {
if !l.shouldLog(severity.ERROR, depth2) {
return
// If GRPC_GO_LOG_VERBOSITY_LEVEL is less restrictive, it prevails.
if i <= l.grpcVerbosityLevel {
return true
}
log.ErrorfDepth(context.TODO(), depth2, format, args...)
}

func (l *grpcLogger) Fatal(args ...interface{}) {
log.FatalfDepth(context.TODO(), depth2, "", args...)
}

func (l *grpcLogger) Fatalln(args ...interface{}) {
log.FatalfDepth(context.TODO(), depth2, "", args...)
}

func (l *grpcLogger) Fatalf(format string, args ...interface{}) {
log.FatalfDepth(context.TODO(), depth2, format, args...)
}

func (l *grpcLogger) V(i int) bool {
return l.vDepth(i, depth2)
}

// https://github.com/grpc/grpc-go/blob/v1.29.1/clientconn.go#L1275
var outgoingConnSpamReSrc = `^grpc: addrConn\.createTransport failed to connect to.*(` +
// *nix
`connection refused` + `|` +
// Windows
`No connection could be made because the target machine actively refused it` + `|` +
// Host removed from the network and no longer resolvable:
// https://github.com/golang/go/blob/go1.8.3/src/net/net.go#L566
`no such host` + `|` +
// RPC dialer is currently failing but also retrying.
regexp.QuoteMeta(errConnectionInterruptedMsg) +
`)`

// When a TCP probe simply opens and immediately closes the
// connection, gRPC is unhappy that the TLS handshake did not
// complete. We don't care.
const incomingConnSpamReSrc = `^grpc: Server\.Serve failed to complete security handshake from "[^"]*": EOF`

var outgoingConnSpamRe = regexp.MustCompile(outgoingConnSpamReSrc)
var incomingConnSpamRe = regexp.MustCompile(incomingConnSpamReSrc)

var spamMu = struct {
syncutil.Mutex
strs map[string]time.Time
}{
strs: make(map[string]time.Time),
// Otherwise, our logger decides.
return log.VDepth(log.Level(i) /* level */, depth+1)
}

const minSpamLogInterval = 30 * time.Second

// shouldPrintWarning returns true iff the gRPC warning message
// identified by args can be logged now. It returns false for outgoing
// connections errors that repeat within minSpamLogInterval of each
// other, and also for certain types of incoming connection errors.
func (l *grpcLogger) shouldPrintWarning(depth int, args ...interface{}) bool {
if l.vDepth(1, depth+1) {
// When verbose logging is on at all, don't
// suppress any warnings.
func (l *grpcLogger) shouldLog(incomingSeverity log.Severity, depth int) bool {
// If the incoming severity is at or above threshold, log.
if l.sev <= incomingSeverity {
return true
}
for _, arg := range args {
if argStr, ok := arg.(string); ok {
// Incoming connection errors that match the criteria are blocked
// always.
if incomingConnSpamRe.MatchString(argStr) {
return false
}

// Outgoing connection errors are only reported if performed too
// often.
// TODO(knz): Maybe the string map should be cleared periodically,
// to avoid unbounded memory growth.
if outgoingConnSpamRe.MatchString(argStr) {
now := timeutil.Now()
spamMu.Lock()
t, ok := spamMu.strs[argStr]
doPrint := !(ok && now.Sub(t) < minSpamLogInterval)
if doPrint {
spamMu.strs[argStr] = now
}
spamMu.Unlock()
return doPrint
}
}
// If verbose logging is on (either for our
// logger or via the grpc env var), log all severities.
i := 4 - int(incomingSeverity)
if i < 0 {
i = 0 // shouldn't happen
}
return true
return l.vDepth(i, depth+1)
}
Loading