diff --git a/docs/generated/logging.md b/docs/generated/logging.md index 7ab607fcef21..f6fce73ab4f7 100644 --- a/docs/generated/logging.md +++ b/docs/generated/logging.md @@ -37,6 +37,31 @@ production deployments are invited to not collect The DEV channel logs in centralized logging facilities, because they likely contain sensitive operational data. +## OPS + +The OPS channel is the channel used to report "point" operational events, +initiated by user operators or automation: + +- operator or system actions on server processes: process starts, + stops, shutdowns, crashes (if they can be logged), + including each time: command-line parameters, current version being run. +- actions that impact the topology of a cluster: node additions, + removals, decommissions, etc. +- job-related initiation or termination. +- cluster setting changes. +- zone configuration changes. + +## HEALTH + +The HEALTH channel is the channel used to report "background" operational +events, initiated by CockroachDB or reporting on automatic processes: + +- current resource usage, including critical resource usage. +- node-node connection events, including connection errors and + gossip details. +- range and table leasing events. +- up-, down-replication; range unavailability. + ## STORAGE The STORAGE channel is the channel used to report low-level storage diff --git a/pkg/cli/demo_cluster.go b/pkg/cli/demo_cluster.go index b20f3c778742..3109b22dc957 100644 --- a/pkg/cli/demo_cluster.go +++ b/pkg/cli/demo_cluster.go @@ -572,7 +572,7 @@ func maybeWarnMemSize(ctx context.Context) { requestedMem := (demoCtx.cacheSize + demoCtx.sqlPoolMemorySize) * int64(demoCtx.nodes) maxRecommendedMem := int64(.75 * float64(maxMemory)) if requestedMem > maxRecommendedMem { - log.Shoutf( + log.Ops.Shoutf( ctx, severity.WARNING, `HIGH MEMORY USAGE The sum of --max-sql-memory (%s) and --cache (%s) multiplied by the diff --git a/pkg/cli/error.go b/pkg/cli/error.go index 78ebb5b55b7f..33c0682674dc 100644 --- a/pkg/cli/error.go +++ b/pkg/cli/error.go @@ -380,7 +380,7 @@ func maybeShoutError( } func checkAndMaybeShout(err error) error { - return checkAndMaybeShoutTo(err, log.Shoutf) + return checkAndMaybeShoutTo(err, log.Ops.Shoutf) } func checkAndMaybeShoutTo( diff --git a/pkg/cli/flags_test.go b/pkg/cli/flags_test.go index 7097224191b8..04403253541d 100644 --- a/pkg/cli/flags_test.go +++ b/pkg/cli/flags_test.go @@ -52,6 +52,7 @@ func TestStdFlagToPflag(t *testing.T) { func TestNoLinkForbidden(t *testing.T) { defer leaktest.AfterTest(t)() defer log.Scope(t).Close(t) + // Verify that the cockroach binary doesn't depend on certain packages. buildutil.VerifyNoImports(t, "github.com/cockroachdb/cockroach/pkg/cmd/cockroach", true, diff --git a/pkg/cli/initial_sql.go b/pkg/cli/initial_sql.go index cb06aa781780..0140ce7e7f29 100644 --- a/pkg/cli/initial_sql.go +++ b/pkg/cli/initial_sql.go @@ -42,10 +42,10 @@ func runInitialSQL( // 1 so as to avoid warning messages and unnecessary rebalance // churn. if err := cliDisableReplication(ctx, s); err != nil { - log.Errorf(ctx, "could not disable replication: %v", err) + log.Ops.Errorf(ctx, "could not disable replication: %v", err) return "", err } - log.Infof(ctx, "Replication was disabled for this cluster.\n"+ + log.Ops.Infof(ctx, "Replication was disabled for this cluster.\n"+ "When/if adding nodes in the future, update zone configurations to increase the replication factor.") } diff --git a/pkg/cli/log_flags.go b/pkg/cli/log_flags.go index e09411c6651f..2d08ee51dba3 100644 --- a/pkg/cli/log_flags.go +++ b/pkg/cli/log_flags.go @@ -181,8 +181,7 @@ func setupLogging(ctx context.Context, cmd *cobra.Command, isServerCmd, applyCon if cliCtx.ambiguousLogDir { // Note that we can't report this message earlier, because the log directory // may not have been ready before the call to MkdirAll() above. - // TODO(knz): send this this to the OPS channel. - log.Dev.Shout(ctx, severity.WARNING, + log.Ops.Shout(ctx, severity.WARNING, "multiple stores configured, "+ "you may want to specify --log='file-defaults: {dir: ...}' to disambiguate.") } @@ -425,8 +424,7 @@ var predefinedLogFiles = map[logpb.Channel]string{ // synchronous writes. Audit logs are configured this way to ensure // non-repudiability. var predefinedAuditFiles = map[logpb.Channel]bool{ - channel.SESSIONS: true, - // FIXME(knz): Add this: - // channel.PRIVILEGES: true, + channel.SESSIONS: true, + channel.PRIVILEGES: true, channel.SENSITIVE_ACCESS: true, } diff --git a/pkg/cli/start.go b/pkg/cli/start.go index 558410f6d9d7..5f0a68c65eb6 100644 --- a/pkg/cli/start.go +++ b/pkg/cli/start.go @@ -422,9 +422,9 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr // If another process was waiting on the PID (e.g. using a FIFO), // this is when we can tell them the node has started listening. if startCtx.pidFile != "" { - log.Infof(ctx, "PID file: %s", startCtx.pidFile) + log.Ops.Infof(ctx, "PID file: %s", startCtx.pidFile) if err := ioutil.WriteFile(startCtx.pidFile, []byte(fmt.Sprintf("%d\n", os.Getpid())), 0644); err != nil { - log.Errorf(ctx, "failed writing the PID: %v", err) + log.Ops.Errorf(ctx, "failed writing the PID: %v", err) } } @@ -438,7 +438,7 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr // the cluster initializes, at which point it will be picked up // and let the client go through, transparently.) if startCtx.listeningURLFile != "" { - log.Infof(ctx, "listening URL file: %s", startCtx.listeningURLFile) + log.Ops.Infof(ctx, "listening URL file: %s", startCtx.listeningURLFile) // (Re-)compute the client connection URL. We cannot do this // earlier (e.g. above, in the runStart function) because // at this time the address and port have not been resolved yet. @@ -450,13 +450,13 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr } if err = ioutil.WriteFile(startCtx.listeningURLFile, []byte(fmt.Sprintf("%s\n", pgURL)), 0644); err != nil { - log.Errorf(ctx, "failed writing the URL: %v", err) + log.Ops.Errorf(ctx, "failed writing the URL: %v", err) } } if waitForInit { - log.Shout(ctx, severity.INFO, - "initial startup completed\n"+ + log.Ops.Shout(ctx, severity.INFO, + "initial startup completed.\n"+ "Node will now attempt to join a running cluster, or wait for `cockroach init`.\n"+ "Client connections will be accepted after this completes successfully.\n"+ "Check the log file(s) for progress. ") @@ -470,7 +470,7 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr // Signal readiness. This unblocks the process when running with // --background or under systemd. if err := sdnotify.Ready(); err != nil { - log.Errorf(ctx, "failed to signal readiness using systemd protocol: %s", err) + log.Ops.Errorf(ctx, "failed to signal readiness using systemd protocol: %s", err) } } @@ -486,11 +486,11 @@ func runStart(cmd *cobra.Command, args []string, startSingleNode bool) (returnEr If problems persist, please see %s.` docLink := docs.URL("cluster-setup-troubleshooting.html") if !startCtx.inBackground { - log.Shoutf(context.Background(), severity.WARNING, msg, docLink) + log.Ops.Shoutf(context.Background(), severity.WARNING, msg, docLink) } else { // Don't shout to stderr since the server will have detached by // the time this function gets called. - log.Warningf(ctx, msg, docLink) + log.Ops.Warningf(ctx, msg, docLink) } } @@ -498,14 +498,14 @@ If problems persist, please see %s.` // We need to make sure this happens before any queries involving geospatial data is executed. loc, err := geos.EnsureInit(geos.EnsureInitErrorDisplayPrivate, startCtx.geoLibsDir) if err != nil { - log.Infof(ctx, "could not initialize GEOS - spatial functions may not be available: %v", err) + log.Ops.Infof(ctx, "could not initialize GEOS - spatial functions may not be available: %v", err) } else { - log.Infof(ctx, "GEOS loaded from directory %s", loc) + log.Ops.Infof(ctx, "GEOS loaded from directory %s", loc) } // Beyond this point, the configuration is set and the server is // ready to start. - log.Info(ctx, "starting cockroach node") + log.Ops.Info(ctx, "starting cockroach node") // Run the rest of the startup process in a goroutine separate from // the main goroutine to avoid preventing proper handling of signals @@ -616,7 +616,7 @@ If problems persist, please see %s.` sCtx := rpc.MakeSecurityContext(serverCfg.Config, security.ClusterTLSSettings(serverCfg.Settings), roachpb.SystemTenantID) pgURL, err := sCtx.PGURL(url.User(security.RootUser)) if err != nil { - log.Errorf(ctx, "failed computing the URL: %v", err) + log.Ops.Errorf(ctx, "failed computing the URL: %v", err) return err } buf.Printf("sql:\t%s\n", pgURL) @@ -681,7 +681,7 @@ If problems persist, please see %s.` return err } msgS := msg.ToString() - log.Infof(ctx, "node startup completed:\n%s", msgS) + log.Ops.Infof(ctx, "node startup completed:\n%s", msgS) if !startCtx.inBackground && !log.LoggingToStderr(severity.INFO) { fmt.Print(msgS.StripMarkers()) } @@ -730,8 +730,7 @@ If problems persist, please see %s.` // timely, and we don't want logs to be lost. log.StartSync() - log.Infof(shutdownCtx, "received signal '%s'", sig) - + log.Ops.Infof(shutdownCtx, "received signal '%s'", sig) switch sig { case os.Interrupt: // Graceful shutdown after an interrupt should cause the process @@ -777,7 +776,7 @@ If problems persist, please see %s.` for { remaining, _, err := s.Drain(drainCtx) if err != nil { - log.Errorf(drainCtx, "graceful drain failed: %v", err) + log.Ops.Errorf(drainCtx, "graceful drain failed: %v", err) break } if remaining == 0 { @@ -815,7 +814,7 @@ If problems persist, please see %s.` // indicates it has stopped. const msgDrain = "initiating graceful shutdown of server" - log.Info(shutdownCtx, msgDrain) + log.Ops.Info(shutdownCtx, msgDrain) fmt.Fprintln(os.Stdout, msgDrain) // Notify the user every 5 second of the shutdown progress. @@ -825,7 +824,7 @@ If problems persist, please see %s.` for { select { case <-ticker.C: - log.Infof(context.Background(), "%d running tasks", stopper.NumTasks()) + log.Ops.Infof(context.Background(), "%d running tasks", stopper.NumTasks()) case <-stopper.ShouldStop(): return case <-stopWithoutDrain: @@ -853,13 +852,13 @@ If problems persist, please see %s.` case termSignal: // Double SIGTERM, or SIGTERM after another signal: continue // the graceful shutdown. - log.Infof(shutdownCtx, "received additional signal '%s'; continuing graceful shutdown", sig) + log.Ops.Infof(shutdownCtx, "received additional signal '%s'; continuing graceful shutdown", sig) continue } // This new signal is not welcome, as it interferes with the graceful // shutdown process. - log.Shoutf(shutdownCtx, severity.ERROR, + log.Ops.Shoutf(shutdownCtx, severity.ERROR, "received signal '%s' during shutdown, initiating hard shutdown%s", log.Safe(sig), log.Safe(hardShutdownHint)) handleSignalDuringShutdown(sig) @@ -867,12 +866,12 @@ If problems persist, please see %s.` case <-stopper.IsStopped(): const msgDone = "server drained and shutdown completed" - log.Infof(shutdownCtx, msgDone) + log.Ops.Infof(shutdownCtx, msgDone) fmt.Fprintln(os.Stdout, msgDone) case <-stopWithoutDrain: const msgDone = "too early to drain; used hard shutdown instead" - log.Infof(shutdownCtx, msgDone) + log.Ops.Infof(shutdownCtx, msgDone) fmt.Fprintln(os.Stdout, msgDone) } break @@ -905,7 +904,7 @@ func hintServerCmdFlags(ctx context.Context, cmd *cobra.Command) { if !listenAddrSpecified && !advAddrSpecified { host, _, _ := net.SplitHostPort(serverCfg.AdvertiseAddr) - log.Shoutf(ctx, severity.WARNING, + log.Ops.Shoutf(ctx, severity.WARNING, "neither --listen-addr nor --advertise-addr was specified.\n"+ "The server will advertise %q to other nodes, is this routable?\n\n"+ "Consider using:\n"+ @@ -930,14 +929,14 @@ func clientFlagsRPC() string { func reportConfiguration(ctx context.Context) { serverCfg.Report(ctx) if envVarsUsed := envutil.GetEnvVarsUsed(); len(envVarsUsed) > 0 { - log.Infof(ctx, "using local environment variables: %s", strings.Join(envVarsUsed, ", ")) + log.Ops.Infof(ctx, "using local environment variables: %s", strings.Join(envVarsUsed, ", ")) } // If a user ever reports "bad things have happened", any // troubleshooting steps will want to rule out that the user was // running as root in a multi-user environment, or using different // uid/gid across runs in the same data directory. To determine // this, it's easier if the information appears in the log file. - log.Infof(ctx, "process identity: %s", sysutil.ProcessIdentity()) + log.Ops.Infof(ctx, "process identity: %s", sysutil.ProcessIdentity()) } func maybeWarnMemorySizes(ctx context.Context) { @@ -952,7 +951,7 @@ func maybeWarnMemorySizes(ctx context.Context) { } else { fmt.Fprintf(&buf, " If you have a dedicated server a reasonable setting is 25%% of physical memory.") } - log.Warningf(ctx, "%s", buf.String()) + log.Ops.Warningf(ctx, "%s", buf.String()) } // Check that the total suggested "max" memory is well below the available memory. @@ -960,7 +959,7 @@ func maybeWarnMemorySizes(ctx context.Context) { requestedMem := serverCfg.CacheSize + serverCfg.MemoryPoolSize maxRecommendedMem := int64(.75 * float64(maxMemory)) if requestedMem > maxRecommendedMem { - log.Shoutf(ctx, severity.WARNING, + log.Ops.Shoutf(ctx, severity.WARNING, "the sum of --max-sql-memory (%s) and --cache (%s) is larger than 75%% of total RAM (%s).\nThis server is running at increased risk of memory-related failures.", sqlSizeValue, cacheSizeValue, humanizeutil.IBytes(maxRecommendedMem)) } @@ -986,7 +985,7 @@ func setupAndInitializeLoggingAndProfiling( if addr == "" { addr = "any of your IP addresses" } - log.Shoutf(context.Background(), severity.WARNING, + log.Ops.Shoutf(context.Background(), severity.WARNING, "ALL SECURITY CONTROLS HAVE BEEN DISABLED!\n\n"+ "This mode is intended for non-production testing only.\n"+ "\n"+ @@ -996,7 +995,7 @@ func setupAndInitializeLoggingAndProfiling( "- Intruders can log in without password and read or write any data in the cluster.\n"+ "- Intruders can consume all your server's resources and cause unavailability.", addr) - log.Shoutf(context.Background(), severity.INFO, + log.Ops.Shoutf(context.Background(), severity.INFO, "To start a secure server without mandating TLS for clients,\n"+ "consider --accept-sql-without-tls instead. For other options, see:\n\n"+ "- %s\n"+ @@ -1011,7 +1010,7 @@ func setupAndInitializeLoggingAndProfiling( // We log build information to stdout (for the short summary), but also // to stderr to coincide with the full logs. info := build.GetInfo() - log.Infof(ctx, "%s", info.Short()) + log.Ops.Infof(ctx, "%s", info.Short()) initCPUProfile(ctx, serverCfg.CPUProfileDirName, serverCfg.Settings) initBlockProfile() diff --git a/pkg/gossip/gossip.go b/pkg/gossip/gossip.go index ac997fa20713..1564b23dbd27 100644 --- a/pkg/gossip/gossip.go +++ b/pkg/gossip/gossip.go @@ -447,7 +447,7 @@ func (g *Gossip) SetStorage(storage Storage) error { // Maintain lock ordering. var storedBI BootstrapInfo if err := storage.ReadBootstrapInfo(&storedBI); err != nil { - log.Warningf(ctx, "failed to read gossip bootstrap info: %s", err) + log.Ops.Warningf(ctx, "failed to read gossip bootstrap info: %s", err) } g.mu.Lock() @@ -492,7 +492,7 @@ func (g *Gossip) SetStorage(storage Storage) error { // If a new resolver was found, immediately signal bootstrap. if newResolverFound { if log.V(1) { - log.Infof(ctx, "found new resolvers from storage; signaling bootstrap") + log.Health.Infof(ctx, "found new resolvers from storage; signaling bootstrap") } g.signalStalledLocked() } @@ -564,7 +564,7 @@ func (g *Gossip) LogStatus() { } ctx := g.AnnotateCtx(context.TODO()) - log.Infof(ctx, "gossip status (%s, %d node%s)\n%s%s%s", + log.Health.Infof(ctx, "gossip status (%s, %d node%s)\n%s%s%s", status, n, util.Pluralize(int64(n)), g.clientStatus(), g.server.status(), connectivity) @@ -688,7 +688,7 @@ func (g *Gossip) maybeAddResolverLocked(addr util.UnresolvedAddr) bool { ctx := g.AnnotateCtx(context.TODO()) r, err := resolver.NewResolverFromUnresolvedAddr(addr) if err != nil { - log.Warningf(ctx, "bad address %s: %s", addr, err) + log.Ops.Warningf(ctx, "bad address %s: %s", addr, err) return false } g.resolvers = append(g.resolvers, r) @@ -814,10 +814,10 @@ func (g *Gossip) updateNodeAddress(key string, content roachpb.Value) { if desc.NodeID == 0 || desc.Address.IsEmpty() { nodeID, err := NodeIDFromKey(key, KeyNodeIDPrefix) if err != nil { - log.Errorf(ctx, "unable to update node address for removed node: %s", err) + log.Health.Errorf(ctx, "unable to update node address for removed node: %s", err) return } - log.Infof(ctx, "removed n%d from gossip", nodeID) + log.Health.Infof(ctx, "removed n%d from gossip", nodeID) g.removeNodeDescriptorLocked(nodeID) return } @@ -1273,7 +1273,7 @@ func (g *Gossip) getNextBootstrapAddressLocked() net.Addr { if addr, err := resolver.GetAddress(); err != nil { if _, ok := g.resolversTried[g.resolverIdx]; !ok { ctx := g.AnnotateCtx(context.TODO()) - log.Warningf(ctx, "invalid bootstrap address: %+v, %v", resolver, err) + log.Ops.Warningf(ctx, "invalid bootstrap address: %+v, %v", resolver, err) } continue } else { @@ -1396,7 +1396,7 @@ func (g *Gossip) manage() { return c.peerID == leastUsefulID }); c != nil { if log.V(1) { - log.Infof(ctx, "closing least useful client %+v to tighten network graph", c) + log.Health.Infof(ctx, "closing least useful client %+v to tighten network graph", c) } log.Eventf(ctx, "culling %s", c.addr) c.close() @@ -1408,7 +1408,7 @@ func (g *Gossip) manage() { } else { if log.V(1) { g.clientsMu.Lock() - log.Infof(ctx, "couldn't find least useful client among %+v", g.clientsMu.clients) + log.Health.Infof(ctx, "couldn't find least useful client among %+v", g.clientsMu.clients) g.clientsMu.Unlock() } } @@ -1447,9 +1447,9 @@ func (g *Gossip) tightenNetwork(ctx context.Context) { return } if nodeAddr, err := g.getNodeIDAddressLocked(distantNodeID); err != nil { - log.Errorf(ctx, "unable to get address for n%d: %s", distantNodeID, err) + log.Health.Errorf(ctx, "unable to get address for n%d: %s", distantNodeID, err) } else { - log.Infof(ctx, "starting client to n%d (%d > %d) to tighten network graph", + log.Health.Infof(ctx, "starting client to n%d (%d > %d) to tighten network graph", distantNodeID, distantHops, maxHops) log.Eventf(ctx, "tightening network with new client to %s", nodeAddr) g.startClientLocked(nodeAddr) @@ -1487,15 +1487,15 @@ func (g *Gossip) maybeSignalStatusChangeLocked() { if orphaned { if len(g.resolvers) == 0 { if log.V(1) { - log.Warningf(ctx, "no resolvers found; use --join to specify a connected node") + log.Ops.Warningf(ctx, "no resolvers found; use --join to specify a connected node") } } else { - log.Warningf(ctx, "no incoming or outgoing connections") + log.Health.Warningf(ctx, "no incoming or outgoing connections") } } else if len(g.resolversTried) == len(g.resolvers) { - log.Warningf(ctx, "first range unavailable; resolvers exhausted") + log.Health.Warningf(ctx, "first range unavailable; resolvers exhausted") } else { - log.Warningf(ctx, "first range unavailable; trying remaining resolvers") + log.Health.Warningf(ctx, "first range unavailable; trying remaining resolvers") } } if len(g.resolvers) > 0 { @@ -1504,7 +1504,7 @@ func (g *Gossip) maybeSignalStatusChangeLocked() { } else { if g.stalled { log.Eventf(ctx, "connected") - log.Infof(ctx, "node has connected to cluster via gossip") + log.Ops.Infof(ctx, "node has connected to cluster via gossip") g.signalConnectedLocked() } g.maybeCleanupBootstrapAddressesLocked() diff --git a/pkg/rpc/breaker.go b/pkg/rpc/breaker.go index 93a745911006..14541a3112d7 100644 --- a/pkg/rpc/breaker.go +++ b/pkg/rpc/breaker.go @@ -106,10 +106,10 @@ type breakerLogger struct { func (r breakerLogger) Debugf(format string, v ...interface{}) { if log.V(2) { - log.InfofDepth(r.ctx, 1, format, v...) + log.Dev.InfofDepth(r.ctx, 1, format, v...) } } func (r breakerLogger) Infof(format string, v ...interface{}) { - log.InfofDepth(r.ctx, 1, format, v...) + log.Ops.InfofDepth(r.ctx, 1, format, v...) } diff --git a/pkg/rpc/clock_offset.go b/pkg/rpc/clock_offset.go index cbbcb586db6f..d3ccffcbd28f 100644 --- a/pkg/rpc/clock_offset.go +++ b/pkg/rpc/clock_offset.go @@ -177,7 +177,7 @@ func (r *RemoteClockMonitor) UpdateOffset( } if log.V(2) { - log.Infof(ctx, "update offset: %s %v", addr, r.mu.offsets[addr]) + log.Health.Infof(ctx, "update offset: %s %v", addr, r.mu.offsets[addr]) } } @@ -232,7 +232,7 @@ func (r *RemoteClockMonitor) VerifyClockOffset(ctx context.Context) error { maxOffset, healthyOffsetCount, numClocks) } if log.V(1) { - log.Infof(ctx, "%d of %d nodes are within the maximum clock offset of %s", healthyOffsetCount, numClocks, maxOffset) + log.Health.Infof(ctx, "%d of %d nodes are within the maximum clock offset of %s", healthyOffsetCount, numClocks, maxOffset) } } @@ -264,7 +264,7 @@ func (r RemoteOffset) isHealthy(ctx context.Context, maxOffset time.Duration) bo // health is ambiguous. For now, we err on the side of not spuriously // killing nodes. if log.V(1) { - log.Infof(ctx, "uncertain remote offset %s for maximum tolerated offset %s, treating as healthy", r, toleratedOffset) + log.Health.Infof(ctx, "uncertain remote offset %s for maximum tolerated offset %s, treating as healthy", r, toleratedOffset) } return true } diff --git a/pkg/rpc/context.go b/pkg/rpc/context.go index 26afb9b2f3c2..63dfb573e445 100644 --- a/pkg/rpc/context.go +++ b/pkg/rpc/context.go @@ -636,12 +636,12 @@ func (ctx *Context) removeConn(conn *Connection, keys ...connKey) { ctx.conns.Delete(key) } if log.V(1) { - log.Infof(ctx.masterCtx, "closing %+v", keys) + log.Health.Infof(ctx.masterCtx, "closing %+v", keys) } if grpcConn := conn.grpcConn; grpcConn != nil { if err := grpcConn.Close(); err != nil && !grpcutil.IsClosedConnection(err) { if log.V(1) { - log.Errorf(ctx.masterCtx, "failed to close client connection: %v", err) + log.Health.Errorf(ctx.masterCtx, "failed to close client connection: %v", err) } } } @@ -956,7 +956,7 @@ func (ctx *Context) grpcDialRaw( dialOpts = append(dialOpts, ctx.testingDialOpts...) if log.V(1) { - log.Infof(ctx.masterCtx, "dialing %s", target) + log.Health.Infof(ctx.masterCtx, "dialing %s", target) } conn, err := grpc.DialContext(ctx.masterCtx, target, dialOpts...) return conn, dialer.redialChan, err @@ -981,7 +981,7 @@ func (ctx *Context) GRPCDialNode( target string, remoteNodeID roachpb.NodeID, class ConnectionClass, ) *Connection { if remoteNodeID == 0 && !ctx.TestingAllowNamedRPCToAnonymousServer { - log.Fatalf(context.TODO(), "invalid node ID 0 in GRPCDialNode()") + log.Fatalf(context.TODO(), "%v", errors.AssertionFailedf("invalid node ID 0 in GRPCDialNode()")) } return ctx.grpcDialNodeInternal(target, remoteNodeID, class) } @@ -1029,7 +1029,7 @@ func (ctx *Context) grpcDialNodeInternal( ctx.Stopper.RunWorker(masterCtx, func(masterCtx context.Context) { err := ctx.runHeartbeat(conn, target, redialChan) if err != nil && !grpcutil.IsClosedConnection(err) { - log.Errorf(masterCtx, "removing connection to %s due to error: %s", target, err) + log.Health.Errorf(masterCtx, "removing connection to %s due to error: %s", target, err) } ctx.removeConn(conn, thisConnKeys...) }) diff --git a/pkg/rpc/context_test.go b/pkg/rpc/context_test.go index c11dfd467e4a..529b08a20fae 100644 --- a/pkg/rpc/context_test.go +++ b/pkg/rpc/context_test.go @@ -651,7 +651,7 @@ func TestHeartbeatHealthTransport(t *testing.T) { if err := stopper.RunAsyncTask(ctx, "busyloop-closer", func(ctx context.Context) { for { if _, err := closeConns(); err != nil { - log.Warningf(ctx, "%v", err) + log.Health.Warningf(ctx, "%v", err) } select { case <-done: diff --git a/pkg/rpc/heartbeat.go b/pkg/rpc/heartbeat.go index 4bce74a8a991..6f957df99df2 100644 --- a/pkg/rpc/heartbeat.go +++ b/pkg/rpc/heartbeat.go @@ -75,7 +75,7 @@ func checkClusterName(clusterName string, peerName string) error { err = errors.Errorf( "local cluster name %q does not match peer cluster name %q", clusterName, peerName) } - log.Shoutf(context.Background(), severity.ERROR, "%v", err) + log.Ops.Shoutf(context.Background(), severity.ERROR, "%v", err) return err } return nil @@ -104,7 +104,7 @@ func checkVersion(ctx context.Context, st *cluster.Settings, peerVersion roachpb // with the requester's address. func (hs *HeartbeatService) Ping(ctx context.Context, args *PingRequest) (*PingResponse, error) { if log.V(2) { - log.Infof(ctx, "received heartbeat: %+v vs local cluster %+v node %+v", args, hs.clusterID, hs.nodeID) + log.Health.Infof(ctx, "received heartbeat: %+v vs local cluster %+v node %+v", args, hs.clusterID, hs.nodeID) } // Check that cluster IDs match. clusterID := hs.clusterID.Get() diff --git a/pkg/rpc/nodedialer/nodedialer.go b/pkg/rpc/nodedialer/nodedialer.go index 96d71d514027..7dd73ee7d793 100644 --- a/pkg/rpc/nodedialer/nodedialer.go +++ b/pkg/rpc/nodedialer/nodedialer.go @@ -157,7 +157,7 @@ func (n *Dialer) dial( defer func() { // Enforce a minimum interval between warnings for failed connections. if err != nil && ctx.Err() == nil && breaker != nil && breaker.ShouldLog() { - log.Infof(ctx, "unable to connect to n%d: %s", nodeID, err) + log.Health.Infof(ctx, "unable to connect to n%d: %s", nodeID, err) } }() conn, err := n.rpcContext.GRPCDialNode(addr.String(), nodeID, class).Connect(ctx) diff --git a/pkg/rpc/stats_handler.go b/pkg/rpc/stats_handler.go index c4c8666c66ee..8788c20efa9f 100644 --- a/pkg/rpc/stats_handler.go +++ b/pkg/rpc/stats_handler.go @@ -127,7 +127,7 @@ func (sh *StatsHandler) TagRPC(ctx context.Context, rti *stats.RPCTagInfo) conte func (sh *StatsHandler) HandleRPC(ctx context.Context, rpcStats stats.RPCStats) { remoteAddr, ok := ctx.Value(remoteAddrKey{}).(string) if !ok { - log.Warningf(ctx, "unable to record stats (%+v); remote addr not found in context", rpcStats) + log.Health.Warningf(ctx, "unable to record stats (%+v); remote addr not found in context", rpcStats) return } // There is a race here, but it's meaningless in practice. Worst diff --git a/pkg/rpc/tls.go b/pkg/rpc/tls.go index c19918983fb6..a0485a8e37a1 100644 --- a/pkg/rpc/tls.go +++ b/pkg/rpc/tls.go @@ -261,34 +261,34 @@ func (ctx *SecurityContext) CheckCertificateAddrs(cctx context.Context) { // with the provided certificate. certInfo := cm.NodeCert() if certInfo.Error != nil { - log.Shoutf(cctx, severity.ERROR, + log.Ops.Shoutf(cctx, severity.ERROR, "invalid node certificate: %v", certInfo.Error) } else { cert := certInfo.ParsedCertificates[0] addrInfo := certAddrs(cert) // Log the certificate details in any case. This will aid during troubleshooting. - log.Infof(cctx, "server certificate addresses: %s", addrInfo) + log.Ops.Infof(cctx, "server certificate addresses: %s", addrInfo) var msg bytes.Buffer // Verify the compatibility. This requires that ValidateAddrs() has // been called already. host, _, err := net.SplitHostPort(ctx.config.AdvertiseAddr) if err != nil { - panic("programming error: call ValidateAddrs() first") + panic(errors.AssertionFailedf("programming error: call ValidateAddrs() first")) } if err := cert.VerifyHostname(host); err != nil { fmt.Fprintf(&msg, "advertise address %q not in node certificate (%s)\n", host, addrInfo) } host, _, err = net.SplitHostPort(ctx.config.SQLAdvertiseAddr) if err != nil { - panic("programming error: call ValidateAddrs() first") + panic(errors.AssertionFailedf("programming error: call ValidateAddrs() first")) } if err := cert.VerifyHostname(host); err != nil { fmt.Fprintf(&msg, "advertise SQL address %q not in node certificate (%s)\n", host, addrInfo) } if msg.Len() > 0 { - log.Shoutf(cctx, severity.WARNING, + log.Ops.Shoutf(cctx, severity.WARNING, "%s"+ "Secure client connections are likely to fail.\n"+ "Consider extending the node certificate or tweak --listen-addr/--advertise-addr/--sql-addr/--advertise-sql-addr.", @@ -309,7 +309,7 @@ func (ctx *SecurityContext) CheckCertificateAddrs(cctx context.Context) { certInfo = cm.NodeCert() } if certInfo.Error != nil { - log.Shoutf(cctx, severity.ERROR, + log.Ops.Shoutf(cctx, severity.ERROR, "invalid UI certificate: %v", certInfo.Error) } else { cert := certInfo.ParsedCertificates[0] @@ -317,7 +317,7 @@ func (ctx *SecurityContext) CheckCertificateAddrs(cctx context.Context) { // Log the certificate details in any case. This will aid during // troubleshooting. - log.Infof(cctx, "web UI certificate addresses: %s", addrInfo) + log.Ops.Infof(cctx, "web UI certificate addresses: %s", addrInfo) } } diff --git a/pkg/server/drain.go b/pkg/server/drain.go index 90151d8a1053..ffa9783259a5 100644 --- a/pkg/server/drain.go +++ b/pkg/server/drain.go @@ -54,13 +54,13 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr doDrain := req.DoDrain - log.Infof(ctx, "drain request received with doDrain = %v, shutdown = %v", doDrain, req.Shutdown) + log.Ops.Infof(ctx, "drain request received with doDrain = %v, shutdown = %v", doDrain, req.Shutdown) res := serverpb.DrainResponse{} if doDrain { remaining, info, err := s.server.Drain(ctx) if err != nil { - log.Errorf(ctx, "drain failed: %v", err) + log.Ops.Errorf(ctx, "drain failed: %v", err) return err } res.DrainRemainingIndicator = remaining @@ -78,7 +78,7 @@ func (s *adminServer) Drain(req *serverpb.DrainRequest, stream serverpb.Admin_Dr if doDrain { // The condition "if doDrain" is because we don't need an info // message for just a probe. - log.Infof(ctx, "drain request completed without server shutdown") + log.Ops.Infof(ctx, "drain request completed without server shutdown") } return nil } @@ -154,9 +154,9 @@ func (s *Server) Drain( comma = ", " } info = redact.RedactableString(descBuf.String()) - log.Infof(ctx, "drain remaining: %d", remaining) + log.Ops.Infof(ctx, "drain remaining: %d", remaining) if info != "" { - log.Infof(ctx, "drain details: %s", info) + log.Ops.Infof(ctx, "drain details: %s", info) } }() diff --git a/pkg/server/node_engine_health.go b/pkg/server/node_engine_health.go index 6496e60ea6c8..ffa128e83af6 100644 --- a/pkg/server/node_engine_health.go +++ b/pkg/server/node_engine_health.go @@ -48,7 +48,7 @@ func (n *Node) startAssertEngineHealth( func guaranteedExitFatal(ctx context.Context, msg string, args ...interface{}) { // NB: log.Shout sets up a timer that guarantees process termination. - log.Shoutf(ctx, severity.FATAL, msg, args...) + log.Health.Shoutf(ctx, severity.FATAL, msg, args...) } func (n *Node) assertEngineHealth( diff --git a/pkg/server/server.go b/pkg/server/server.go index 7c3e1ac98e42..48423ea55243 100644 --- a/pkg/server/server.go +++ b/pkg/server/server.go @@ -303,7 +303,7 @@ func NewServer(cfg Config, stopper *stop.Stopper) (*Server, error) { rpcContext.HeartbeatCB = func() { if err := rpcContext.RemoteClocks.VerifyClockOffset(ctx); err != nil { - log.Fatalf(ctx, "%v", err) + log.Ops.Fatalf(ctx, "%v", err) } } registry.AddMetricStruct(rpcContext.Metrics()) @@ -848,7 +848,7 @@ func (s *Server) startMonitoringForwardClockJumps(ctx context.Context) error { return errors.Wrap(err, "monitoring forward clock jumps") } - log.Info(ctx, "monitoring forward clock jumps based on server.clock.forward_jump_check_enabled") + log.Ops.Info(ctx, "monitoring forward clock jumps based on server.clock.forward_jump_check_enabled") return nil } @@ -894,7 +894,7 @@ func ensureClockMonotonicity( currentWallTime := currentWallTimeFn() delta := time.Duration(sleepUntil - currentWallTime) if delta > 0 { - log.Infof( + log.Ops.Infof( ctx, "Sleeping till wall time %v to catches up to %v to ensure monotonicity. Delta: %v", currentWallTime, @@ -938,7 +938,7 @@ func periodicallyPersistHLCUpperBound( persistHLCUpperBoundFn, int64(persistInterval*3), /* delta to compute upper bound */ ); err != nil { - log.Fatalf( + log.Ops.Fatalf( context.Background(), "error persisting HLC upper bound: %v", err, @@ -957,16 +957,16 @@ func periodicallyPersistHLCUpperBound( if persistInterval > 0 { ticker = tickerFn(persistInterval) persistHLCUpperBound() - log.Info(context.Background(), "persisting HLC upper bound is enabled") + log.Ops.Info(context.Background(), "persisting HLC upper bound is enabled") } else { if err := clock.ResetHLCUpperBound(persistHLCUpperBoundFn); err != nil { - log.Fatalf( + log.Ops.Fatalf( context.Background(), "error resetting hlc upper bound: %v", err, ) } - log.Info(context.Background(), "persisting HLC upper bound is disabled") + log.Ops.Info(context.Background(), "persisting HLC upper bound is disabled") } case <-ticker.C: @@ -1291,7 +1291,7 @@ func (s *Server) PreStart(ctx context.Context) error { s.stopper.RunWorker(workersCtx, func(workersCtx context.Context) { <-s.stopper.ShouldQuiesce() if err := conn.Close(); err != nil { - log.Fatalf(workersCtx, "%v", err) + log.Ops.Fatalf(workersCtx, "%v", err) } }) @@ -1460,10 +1460,10 @@ func (s *Server) PreStart(ctx context.Context) error { // but this gossip only happens once the first range has a leaseholder, i.e. // when a quorum of nodes has gone fully operational. _ = s.stopper.RunAsyncTask(ctx, "connect-gossip", func(ctx context.Context) { - log.Infof(ctx, "connecting to gossip network to verify cluster ID %q", state.clusterID) + log.Ops.Infof(ctx, "connecting to gossip network to verify cluster ID %q", state.clusterID) select { case <-s.gossip.Connected: - log.Infof(ctx, "node connected via gossip") + log.Ops.Infof(ctx, "node connected via gossip") case <-ctx.Done(): case <-s.stopper.ShouldQuiesce(): } @@ -1590,15 +1590,15 @@ func (s *Server) PreStart(ctx context.Context) error { // stores) s.node.waitForAdditionalStoreInit() - log.Infof(ctx, "starting %s server at %s (use: %s)", + log.Ops.Infof(ctx, "starting %s server at %s (use: %s)", redact.Safe(s.cfg.HTTPRequestScheme()), s.cfg.HTTPAddr, s.cfg.HTTPAdvertiseAddr) rpcConnType := redact.SafeString("grpc/postgres") if s.cfg.SplitListenSQL { rpcConnType = "grpc" - log.Infof(ctx, "starting postgres server at %s (use: %s)", s.cfg.SQLAddr, s.cfg.SQLAdvertiseAddr) + log.Ops.Infof(ctx, "starting postgres server at %s (use: %s)", s.cfg.SQLAddr, s.cfg.SQLAdvertiseAddr) } - log.Infof(ctx, "starting %s server at %s", rpcConnType, s.cfg.Addr) - log.Infof(ctx, "advertising CockroachDB node at %s", s.cfg.AdvertiseAddr) + log.Ops.Infof(ctx, "starting %s server at %s", rpcConnType, s.cfg.Addr) + log.Ops.Infof(ctx, "advertising CockroachDB node at %s", s.cfg.AdvertiseAddr) log.Event(ctx, "accepting connections") @@ -1613,7 +1613,7 @@ func (s *Server) PreStart(ctx context.Context) error { if err := s.node.stores.VisitStores(func(s *kvserver.Store) error { return s.WriteLastUpTimestamp(ctx, now) }); err != nil { - log.Warningf(ctx, "writing last up timestamp: %v", err) + log.Ops.Warningf(ctx, "writing last up timestamp: %v", err) } }, }) @@ -1703,7 +1703,7 @@ func (s *Server) PreStart(ctx context.Context) error { http.Error(w, "admin privilege required", http.StatusUnauthorized) return } else if err != nil { - log.Infof(authCtx, "web session error: %s", err) + log.Ops.Infof(authCtx, "web session error: %s", err) http.Error(w, "error checking authentication", http.StatusInternalServerError) return } @@ -1815,7 +1815,7 @@ func (s *Server) startListenRPCAndSQL( s.stopper.RunWorker(workersCtx, func(workersCtx context.Context) { <-s.stopper.ShouldQuiesce() if err := pgL.Close(); err != nil { - log.Fatalf(workersCtx, "%v", err) + log.Ops.Fatalf(workersCtx, "%v", err) } }) log.Eventf(ctx, "listening on sql port %s", s.cfg.SQLAddr) @@ -1896,7 +1896,7 @@ func (s *Server) startServeUI( s.stopper.RunWorker(workersCtx, func(workersCtx context.Context) { <-s.stopper.ShouldQuiesce() if err := httpLn.Close(); err != nil { - log.Fatalf(workersCtx, "%v", err) + log.Ops.Fatalf(workersCtx, "%v", err) } }) @@ -1948,7 +1948,7 @@ func (s *sqlServer) startServeSQL( pgL net.Listener, socketFile string, ) error { - log.Info(ctx, "serving sql connections") + log.Ops.Info(ctx, "serving sql connections") // Start servicing SQL connections. pgCtx := s.pgServer.AmbientCtx.AnnotateCtx(context.Background()) @@ -1962,14 +1962,14 @@ func (s *sqlServer) startServeSQL( tcpKeepAlive.configure(connCtx, conn) if err := s.pgServer.ServeConn(connCtx, conn, pgwire.SocketTCP); err != nil { - log.Errorf(connCtx, "serving SQL client conn: %v", err) + log.Ops.Errorf(connCtx, "serving SQL client conn: %v", err) } })) }) // If a unix socket was requested, start serving there too. if len(socketFile) != 0 { - log.Infof(ctx, "starting postgres server at unix:%s", socketFile) + log.Ops.Infof(ctx, "starting postgres server at unix:%s", socketFile) // Unix socket enabled: postgres protocol only. unixLn, err := net.Listen("unix", socketFile) @@ -1980,7 +1980,7 @@ func (s *sqlServer) startServeSQL( stopper.RunWorker(ctx, func(workersCtx context.Context) { <-stopper.ShouldQuiesce() if err := unixLn.Close(); err != nil { - log.Fatalf(workersCtx, "%v", err) + log.Ops.Fatalf(workersCtx, "%v", err) } }) @@ -1988,7 +1988,7 @@ func (s *sqlServer) startServeSQL( netutil.FatalIfUnexpected(connManager.ServeWith(pgCtx, stopper, unixLn, func(conn net.Conn) { connCtx := logtags.AddTag(pgCtx, "client", conn.RemoteAddr().String()) if err := s.pgServer.ServeConn(connCtx, conn, pgwire.SocketUnix); err != nil { - log.Errorf(connCtx, "%v", err) + log.Ops.Errorf(connCtx, "%v", err) } })) }) @@ -2043,7 +2043,7 @@ func (s *Server) Decommission( ctx, txn, eventType, int32(nodeID), int32(s.NodeID()), struct{}{}, ) }); err != nil { - log.Errorf(ctx, "unable to record %s event for node %d: %s", eventType, nodeID, err) + log.Ops.Errorf(ctx, "unable to record %s event for node %d: %s", eventType, nodeID, err) } } } @@ -2214,7 +2214,7 @@ func (s *Server) ServeHTTP(w http.ResponseWriter, r *http.Request) { // which results in a 304 Not Modified. if err := gzw.Close(); err != nil && !errors.Is(err, http.ErrBodyNotAllowed) { ctx := s.AnnotateCtx(r.Context()) - log.Warningf(ctx, "error closing gzip response writer: %v", err) + log.Ops.Warningf(ctx, "error closing gzip response writer: %v", err) } }() w = gzw @@ -2312,14 +2312,14 @@ func (k *tcpKeepAliveManager) configure(ctx context.Context, conn net.Conn) { doLog := atomic.CompareAndSwapInt32(&k.loggedKeepAliveStatus, 0, 1) if err := tcpConn.SetKeepAlive(true); err != nil { if doLog { - log.Warningf(ctx, "failed to enable TCP keep-alive for pgwire: %v", err) + log.Ops.Warningf(ctx, "failed to enable TCP keep-alive for pgwire: %v", err) } return } if err := tcpConn.SetKeepAlivePeriod(k.tcpKeepAlive); err != nil { if doLog { - log.Warningf(ctx, "failed to set TCP keep-alive duration for pgwire: %v", err) + log.Ops.Warningf(ctx, "failed to set TCP keep-alive duration for pgwire: %v", err) } return } diff --git a/pkg/server/status/runtime.go b/pkg/server/status/runtime.go index e2e79e24bb2c..e7518c0e1186 100644 --- a/pkg/server/status/runtime.go +++ b/pkg/server/status/runtime.go @@ -326,11 +326,11 @@ func NewRuntimeStatSampler(ctx context.Context, clock *hlc.Clock) *RuntimeStatSa diskCounters, err := getSummedDiskCounters(ctx) if err != nil { - log.Errorf(ctx, "could not get initial disk IO counters: %v", err) + log.Ops.Errorf(ctx, "could not get initial disk IO counters: %v", err) } netCounters, err := getSummedNetStats(ctx) if err != nil { - log.Errorf(ctx, "could not get initial disk IO counters: %v", err) + log.Ops.Errorf(ctx, "could not get initial disk IO counters: %v", err) } rsr := &RuntimeStatSampler{ @@ -435,11 +435,11 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( pid := os.Getpid() mem := gosigar.ProcMem{} if err := mem.Get(pid); err != nil { - log.Errorf(ctx, "unable to get mem usage: %v", err) + log.Health.Errorf(ctx, "unable to get mem usage: %v", err) } cpuTime := gosigar.ProcTime{} if err := cpuTime.Get(pid); err != nil { - log.Errorf(ctx, "unable to get cpu usage: %v", err) + log.Health.Errorf(ctx, "unable to get cpu usage: %v", err) } cgroupCPU, _ := cgroups.GetCgroupCPU() cpuShare := cgroupCPU.CPUShares() @@ -449,17 +449,17 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( if gosigar.IsNotImplemented(err) { if !rsr.fdUsageNotImplemented { rsr.fdUsageNotImplemented = true - log.Warningf(ctx, "unable to get file descriptor usage (will not try again): %s", err) + log.Health.Warningf(ctx, "unable to get file descriptor usage (will not try again): %s", err) } } else { - log.Errorf(ctx, "unable to get file descriptor usage: %s", err) + log.Health.Errorf(ctx, "unable to get file descriptor usage: %s", err) } } var deltaDisk diskStats diskCounters, err := getSummedDiskCounters(ctx) if err != nil { - log.Warningf(ctx, "problem fetching disk stats: %s; disk stats will be empty.", err) + log.Health.Warningf(ctx, "problem fetching disk stats: %s; disk stats will be empty.", err) } else { deltaDisk = diskCounters subtractDiskCounters(&deltaDisk, rsr.last.disk) @@ -480,7 +480,7 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( var deltaNet net.IOCountersStat netCounters, err := getSummedNetStats(ctx) if err != nil { - log.Warningf(ctx, "problem fetching net stats: %s; net stats will be empty.", err) + log.Health.Warningf(ctx, "problem fetching net stats: %s; net stats will be empty.", err) } else { deltaNet = netCounters subtractNetworkCounters(&deltaNet, rsr.last.net) @@ -522,7 +522,7 @@ func (rsr *RuntimeStatSampler) SampleEnvironment( // TODO(knz): make utility wrapper around humanize.IBytes that // returns a safe value and collapse the entire log.Infof -> Safe -> // Sprintf sequence as a flat Infof call. - log.Infof(ctx, "%s", redact.Safe(fmt.Sprintf("runtime stats: %s RSS, %d goroutines, %s/%s/%s GO alloc/idle/total%s, "+ + log.Health.Infof(ctx, "%s", redact.Safe(fmt.Sprintf("runtime stats: %s RSS, %d goroutines, %s/%s/%s GO alloc/idle/total%s, "+ "%s/%s CGO alloc/total, %.1f CGO/sec, %.1f/%.1f %%(u/s)time, %.1f %%gc (%dx), "+ "%s/%s (r/w)net", humanize.IBytes(mem.Resident), numGoroutine, diff --git a/pkg/util/log/channel/channel_generated.go b/pkg/util/log/channel/channel_generated.go index 7e0f63660ce9..58d522792b79 100644 --- a/pkg/util/log/channel/channel_generated.go +++ b/pkg/util/log/channel/channel_generated.go @@ -16,6 +16,29 @@ import "github.com/cockroachdb/cockroach/pkg/util/log/logpb" // sensitive operational data. const DEV = logpb.Channel_DEV +// OPS is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +const OPS = logpb.Channel_OPS + +// HEALTH is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +const HEALTH = logpb.Channel_HEALTH + // STORAGE is the channel used to report low-level storage // layer events (RocksDB/Pebble). const STORAGE = logpb.Channel_STORAGE diff --git a/pkg/util/log/clog.go b/pkg/util/log/clog.go index 7b5c7f9d79e0..126bef212f34 100644 --- a/pkg/util/log/clog.go +++ b/pkg/util/log/clog.go @@ -205,7 +205,7 @@ func SetClusterID(clusterID string) { // new log files, even on the first log file. This ensures that grep // will always find it. ctx := logtags.AddTag(context.Background(), "config", nil) - logfDepth(ctx, 1, severity.INFO, channel.DEV, "clusterID: %s", clusterID) // TODO(knz): Use OPS here. + logfDepth(ctx, 1, severity.INFO, channel.OPS, "clusterID: %s", clusterID) // Perform the change proper. logging.mu.Lock() diff --git a/pkg/util/log/doc.go b/pkg/util/log/doc.go index 213b77364840..784659c470e5 100644 --- a/pkg/util/log/doc.go +++ b/pkg/util/log/doc.go @@ -31,13 +31,25 @@ // Named Functions // // The functions Info, Warning, Error, and Fatal log their arguments at the -// specified level. All include formatting variants like Infof. +// severity level. All include formatting variants like Infof. // // Examples: // -// log.Info(ctx, "prepare to repel boarders") -// log.Fatal(ctx, "initialization failed", err) -// log.Infof(ctx, "client error: %s", err) +// log.Dev.Info(ctx, "prepare to repel boarders") +// log.Ops.Info(ctx, "prepare to repel boarders") +// log.Dev.Fatal(ctx, "initialization failed", err) +// log.Dev.Infof(ctx, "client error: %s", err) +// +// In these examples, the first word "Dev", "Ops" is the logging +// channel. Different channels are aimed at different audiences and +// can be redirected to different sinks. +// +// If the channel name is omitted (e.g. log.Info), the Dev channel is +// used. +// +// The second word e.g. "Info", "Fatal" etc is the severity level. +// For any given channel, users can filter out logging to a given +// severity level or higher. // // V-Style // @@ -52,10 +64,17 @@ // log.Info(ctx, "starting transaction...") // } // +// Additionally, severity functions also exist in a V variant for +// convenience. For example: +// +// log.Ops.VWarningf(ctx, 2, "attention!") +// +// aliases: if V(2) { log.Ops.Warningf(ctx, "attention!") } +// // Events // // The Event functions log messages to an existing trace if one exists. The -// VEvent functions logs the message to a trace and also the log file based +// VEvent functions logs the message to a trace and also the Dev channel based // on the V level. // // Examples: @@ -96,8 +115,4 @@ // --vmodule=gopher*=3 // sets the V level to 3 in all Go files whose names begin "gopher". // -// Protobuf -// -// Autogenerated: -// package log diff --git a/pkg/util/log/exit_override.go b/pkg/util/log/exit_override.go index 642e45ccf37d..c60625a5399b 100644 --- a/pkg/util/log/exit_override.go +++ b/pkg/util/log/exit_override.go @@ -78,7 +78,7 @@ func (l *loggerT) exitLocked(err error, code exit.Code) { func (l *loggerT) reportErrorEverywhereLocked(ctx context.Context, err error) { // Make a valid log entry for this error. entry := MakeEntry( - ctx, severity.ERROR, channel.DEV, // FIXME(knz): use OPS here. + ctx, severity.ERROR, channel.OPS, 2 /* depth */, true, /* redactable */ "logging error: %v", err) diff --git a/pkg/util/log/log_channels_generated.go b/pkg/util/log/log_channels_generated.go index a32d126213d1..f3bb69fa54fa 100644 --- a/pkg/util/log/log_channels_generated.go +++ b/pkg/util/log/log_channels_generated.go @@ -939,6 +939,833 @@ func Shoutf(ctx context.Context, sev Severity, format string, args ...interface{ shoutfDepth(ctx, 1, sev, channel.DEV, format, args...) } +// loggerOps is the logger type for the OPS channel. +type loggerOps struct{} + +// Ops is a logger that logs to the OPS channel. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +var Ops loggerOps + +// Ops and loggerOps implement ChannelLogger. +// +// We do not force use of ChannelLogger when instantiating the logger +// object above (e.g. by giving it the interface type), to ensure +// the calls to the API methods remain inlinable in the common case. +var _ ChannelLogger = Ops + +// Infof logs to the OPS channel with severity INFO. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerOps) Infof(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.INFO, channel.OPS, format, args...) +} + +// VInfof logs to the OPS channel with severity INFO, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerOps) VInfof(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.INFO, channel.OPS, format, args...) + } +} + +// Info logs to the OPS channel with severity INFO. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerOps) Info(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.INFO, channel.OPS, msg) +} + +// InfofDepth logs to the OPS channel with severity INFO, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerOps) InfofDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.INFO, channel.OPS, format, args...) +} + +// Warningf logs to the OPS channel with severity WARNING. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerOps) Warningf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.WARNING, channel.OPS, format, args...) +} + +// VWarningf logs to the OPS channel with severity WARNING, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerOps) VWarningf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.WARNING, channel.OPS, format, args...) + } +} + +// Warning logs to the OPS channel with severity WARNING. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerOps) Warning(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.WARNING, channel.OPS, msg) +} + +// WarningfDepth logs to the OPS channel with severity WARNING, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerOps) WarningfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.WARNING, channel.OPS, format, args...) +} + +// Errorf logs to the OPS channel with severity ERROR. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerOps) Errorf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.ERROR, channel.OPS, format, args...) +} + +// VErrorf logs to the OPS channel with severity ERROR, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerOps) VErrorf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.ERROR, channel.OPS, format, args...) + } +} + +// Error logs to the OPS channel with severity ERROR. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerOps) Error(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.ERROR, channel.OPS, msg) +} + +// ErrorfDepth logs to the OPS channel with severity ERROR, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerOps) ErrorfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.ERROR, channel.OPS, format, args...) +} + +// Fatalf logs to the OPS channel with severity FATAL. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerOps) Fatalf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.FATAL, channel.OPS, format, args...) +} + +// VFatalf logs to the OPS channel with severity FATAL, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerOps) VFatalf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.FATAL, channel.OPS, format, args...) + } +} + +// Fatal logs to the OPS channel with severity FATAL. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerOps) Fatal(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.FATAL, channel.OPS, msg) +} + +// FatalfDepth logs to the OPS channel with severity FATAL, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerOps) FatalfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.FATAL, channel.OPS, format, args...) +} + +// Shout logs to channel OPS, and also to the real stderr if logging +// is currently redirected to a file. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +func (loggerOps) Shout(ctx context.Context, sev Severity, msg string) { + shoutfDepth(ctx, 1, sev, channel.OPS, msg) +} + +// Shoutf logs to channel OPS, and also to the real stderr if +// logging is currently redirected to a file. Arguments are handled in +// the manner of fmt.Printf. +// +// The OPS channel is the channel used to report "point" operational events, +// initiated by user operators or automation: +// +// - operator or system actions on server processes: process starts, +// stops, shutdowns, crashes (if they can be logged), +// including each time: command-line parameters, current version being run. +// - actions that impact the topology of a cluster: node additions, +// removals, decommissions, etc. +// - job-related initiation or termination. +// - cluster setting changes. +// - zone configuration changes. +func (loggerOps) Shoutf(ctx context.Context, sev Severity, format string, args ...interface{}) { + shoutfDepth(ctx, 1, sev, channel.OPS, format, args...) +} + +// loggerHealth is the logger type for the HEALTH channel. +type loggerHealth struct{} + +// Health is a logger that logs to the HEALTH channel. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +var Health loggerHealth + +// Health and loggerHealth implement ChannelLogger. +// +// We do not force use of ChannelLogger when instantiating the logger +// object above (e.g. by giving it the interface type), to ensure +// the calls to the API methods remain inlinable in the common case. +var _ ChannelLogger = Health + +// Infof logs to the HEALTH channel with severity INFO. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerHealth) Infof(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.INFO, channel.HEALTH, format, args...) +} + +// VInfof logs to the HEALTH channel with severity INFO, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerHealth) VInfof(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.INFO, channel.HEALTH, format, args...) + } +} + +// Info logs to the HEALTH channel with severity INFO. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerHealth) Info(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.INFO, channel.HEALTH, msg) +} + +// InfofDepth logs to the HEALTH channel with severity INFO, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The INFO severity is used for informational messages, when no action +// is required as a result. +func (loggerHealth) InfofDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.INFO, channel.HEALTH, format, args...) +} + +// Warningf logs to the HEALTH channel with severity WARNING. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerHealth) Warningf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.WARNING, channel.HEALTH, format, args...) +} + +// VWarningf logs to the HEALTH channel with severity WARNING, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerHealth) VWarningf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.WARNING, channel.HEALTH, format, args...) + } +} + +// Warning logs to the HEALTH channel with severity WARNING. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerHealth) Warning(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.WARNING, channel.HEALTH, msg) +} + +// WarningfDepth logs to the HEALTH channel with severity WARNING, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The WARNING severity is used for situations which may require special handling, +// while normal operation is expected to resume automatically. +func (loggerHealth) WarningfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.WARNING, channel.HEALTH, format, args...) +} + +// Errorf logs to the HEALTH channel with severity ERROR. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerHealth) Errorf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.ERROR, channel.HEALTH, format, args...) +} + +// VErrorf logs to the HEALTH channel with severity ERROR, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerHealth) VErrorf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.ERROR, channel.HEALTH, format, args...) + } +} + +// Error logs to the HEALTH channel with severity ERROR. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerHealth) Error(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.ERROR, channel.HEALTH, msg) +} + +// ErrorfDepth logs to the HEALTH channel with severity ERROR, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The ERROR severity is used for situations that require special handling, +// when normal operation could not proceed as expected. +// Other operations can continue mostly unaffected. +func (loggerHealth) ErrorfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.ERROR, channel.HEALTH, format, args...) +} + +// Fatalf logs to the HEALTH channel with severity FATAL. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerHealth) Fatalf(ctx context.Context, format string, args ...interface{}) { + logfDepth(ctx, 1, severity.FATAL, channel.HEALTH, format, args...) +} + +// VFatalf logs to the HEALTH channel with severity FATAL, +// if logging has been enabled for the source file where the call is +// performed at the provided verbosity level, via the vmodule setting. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerHealth) VFatalf(ctx context.Context, level Level, format string, args ...interface{}) { + if VDepth(level, 1) { + logfDepth(ctx, 1, severity.FATAL, channel.HEALTH, format, args...) + } +} + +// Fatal logs to the HEALTH channel with severity FATAL. +// It extracts log tags from the context and logs them along with the given +// message. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerHealth) Fatal(ctx context.Context, msg string) { + logfDepth(ctx, 1, severity.FATAL, channel.HEALTH, msg) +} + +// FatalfDepth logs to the HEALTH channel with severity FATAL, +// offsetting the caller's stack frame by 'depth'. +// It extracts log tags from the context and logs them along with the given +// message. Arguments are handled in the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +// +// The FATAL severity is used for situations that require an immedate, hard +// server shutdown. A report is also sent to telemetry if telemetry +// is enabled. +func (loggerHealth) FatalfDepth(ctx context.Context, depth int, format string, args ...interface{}) { + logfDepth(ctx, depth+1, severity.FATAL, channel.HEALTH, format, args...) +} + +// Shout logs to channel HEALTH, and also to the real stderr if logging +// is currently redirected to a file. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +func (loggerHealth) Shout(ctx context.Context, sev Severity, msg string) { + shoutfDepth(ctx, 1, sev, channel.HEALTH, msg) +} + +// Shoutf logs to channel HEALTH, and also to the real stderr if +// logging is currently redirected to a file. Arguments are handled in +// the manner of fmt.Printf. +// +// The HEALTH channel is the channel used to report "background" operational +// events, initiated by CockroachDB or reporting on automatic processes: +// +// - current resource usage, including critical resource usage. +// - node-node connection events, including connection errors and +// gossip details. +// - range and table leasing events. +// - up-, down-replication; range unavailability. +func (loggerHealth) Shoutf(ctx context.Context, sev Severity, format string, args ...interface{}) { + shoutfDepth(ctx, 1, sev, channel.HEALTH, format, args...) +} + // loggerStorage is the logger type for the STORAGE channel. type loggerStorage struct{} diff --git a/pkg/util/log/logcrash/crash_reporting.go b/pkg/util/log/logcrash/crash_reporting.go index 1b8d56688fb5..34fbc6b42458 100644 --- a/pkg/util/log/logcrash/crash_reporting.go +++ b/pkg/util/log/logcrash/crash_reporting.go @@ -137,7 +137,7 @@ func ReportPanic(ctx context.Context, sv *settings.Values, r interface{}, depth // sure, whether some other caller further in the call stack is // catching the panic object in the end or not. panicErr := PanicAsError(depth+1, r) - log.Shoutf(ctx, severity.ERROR, "a panic has occurred!\n%+v", panicErr) + log.Ops.Shoutf(ctx, severity.ERROR, "a panic has occurred!\n%+v", panicErr) // In addition to informing the user, also report the details to telemetry. sendCrashReport(ctx, sv, panicErr, ReportTypePanic) diff --git a/pkg/util/log/logpb/log.pb.go b/pkg/util/log/logpb/log.pb.go index b61488340371..7b731cced529 100644 --- a/pkg/util/log/logpb/log.pb.go +++ b/pkg/util/log/logpb/log.pb.go @@ -77,7 +77,7 @@ func (x Severity) String() string { return proto.EnumName(Severity_name, int32(x)) } func (Severity) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_log_db174d54ef3dbcca, []int{0} + return fileDescriptor_log_8a14f28d7f50d5e3, []int{0} } // Channel is the logical logging channel on which a message is sent. @@ -101,6 +101,27 @@ const ( // centralized logging facilities, because they likely contain // sensitive operational data. Channel_DEV Channel = 0 + // OPS is the channel used to report "point" operational events, + // initiated by user operators or automation: + // + // - operator or system actions on server processes: process starts, + // stops, shutdowns, crashes (if they can be logged), + // including each time: command-line parameters, current version being run. + // - actions that impact the topology of a cluster: node additions, + // removals, decommissions, etc. + // - job-related initiation or termination. + // - cluster setting changes. + // - zone configuration changes. + Channel_OPS Channel = 1 + // HEALTH is the channel used to report "background" operational + // events, initiated by CockroachDB or reporting on automatic processes: + // + // - current resource usage, including critical resource usage. + // - node-node connection events, including connection errors and + // gossip details. + // - range and table leasing events. + // - up-, down-replication; range unavailability. + Channel_HEALTH Channel = 2 // STORAGE is the channel used to report low-level storage // layer events (RocksDB/Pebble). Channel_STORAGE Channel = 3 @@ -148,6 +169,8 @@ const ( var Channel_name = map[int32]string{ 0: "DEV", + 1: "OPS", + 2: "HEALTH", 3: "STORAGE", 4: "SESSIONS", 7: "SENSITIVE_ACCESS", @@ -157,6 +180,8 @@ var Channel_name = map[int32]string{ } var Channel_value = map[string]int32{ "DEV": 0, + "OPS": 1, + "HEALTH": 2, "STORAGE": 3, "SESSIONS": 4, "SENSITIVE_ACCESS": 7, @@ -169,7 +194,7 @@ func (x Channel) String() string { return proto.EnumName(Channel_name, int32(x)) } func (Channel) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_log_db174d54ef3dbcca, []int{1} + return fileDescriptor_log_8a14f28d7f50d5e3, []int{1} } // Entry represents a cockroach structured log entry. @@ -214,7 +239,7 @@ func (m *Entry) Reset() { *m = Entry{} } func (m *Entry) String() string { return proto.CompactTextString(m) } func (*Entry) ProtoMessage() {} func (*Entry) Descriptor() ([]byte, []int) { - return fileDescriptor_log_db174d54ef3dbcca, []int{0} + return fileDescriptor_log_8a14f28d7f50d5e3, []int{0} } func (m *Entry) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -253,7 +278,7 @@ func (m *FileDetails) Reset() { *m = FileDetails{} } func (m *FileDetails) String() string { return proto.CompactTextString(m) } func (*FileDetails) ProtoMessage() {} func (*FileDetails) Descriptor() ([]byte, []int) { - return fileDescriptor_log_db174d54ef3dbcca, []int{1} + return fileDescriptor_log_8a14f28d7f50d5e3, []int{1} } func (m *FileDetails) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -289,7 +314,7 @@ func (m *FileInfo) Reset() { *m = FileInfo{} } func (m *FileInfo) String() string { return proto.CompactTextString(m) } func (*FileInfo) ProtoMessage() {} func (*FileInfo) Descriptor() ([]byte, []int) { - return fileDescriptor_log_db174d54ef3dbcca, []int{2} + return fileDescriptor_log_8a14f28d7f50d5e3, []int{2} } func (m *FileInfo) XXX_Unmarshal(b []byte) error { return m.Unmarshal(b) @@ -1295,49 +1320,50 @@ var ( ErrIntOverflowLog = fmt.Errorf("proto: integer overflow") ) -func init() { proto.RegisterFile("util/log/logpb/log.proto", fileDescriptor_log_db174d54ef3dbcca) } +func init() { proto.RegisterFile("util/log/logpb/log.proto", fileDescriptor_log_8a14f28d7f50d5e3) } -var fileDescriptor_log_db174d54ef3dbcca = []byte{ - // 651 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0xcd, 0x6a, 0xdb, 0x4a, - 0x14, 0xf6, 0x58, 0x92, 0x25, 0x9d, 0x84, 0x30, 0x77, 0xc8, 0x05, 0xdd, 0x9b, 0x5c, 0xc5, 0x98, - 0x0b, 0x35, 0x59, 0x38, 0x90, 0x52, 0xe8, 0xae, 0xf8, 0x47, 0x0e, 0x6a, 0x83, 0x9c, 0x8e, 0x9c, - 0xa4, 0xb4, 0x0b, 0x23, 0xdb, 0x13, 0x45, 0x54, 0xd2, 0x18, 0x49, 0x2e, 0xa4, 0xef, 0xd0, 0xd2, - 0x87, 0xe8, 0x53, 0xf4, 0x09, 0xb2, 0xcc, 0x32, 0xab, 0xd0, 0x3a, 0x2f, 0x52, 0x66, 0x64, 0xb5, - 0x81, 0x66, 0x21, 0xf1, 0x9d, 0x4f, 0xdf, 0x39, 0x73, 0xce, 0x77, 0x34, 0x60, 0x2d, 0x8b, 0x28, - 0x3e, 0x88, 0x79, 0x28, 0x9e, 0xc5, 0x54, 0xbc, 0x3b, 0x8b, 0x8c, 0x17, 0x9c, 0x90, 0x19, 0x9f, - 0xbd, 0xcf, 0x78, 0x30, 0xbb, 0xec, 0x08, 0x4d, 0x27, 0xe6, 0xe1, 0xbf, 0xdb, 0x21, 0x0f, 0xb9, - 0xfc, 0x7c, 0x20, 0x50, 0xa9, 0x6c, 0x7d, 0xab, 0x83, 0xe6, 0xa4, 0x45, 0x76, 0x45, 0x9e, 0x83, - 0x91, 0xb3, 0x0f, 0x2c, 0x8b, 0x8a, 0x2b, 0x0b, 0x35, 0x51, 0x7b, 0xeb, 0x70, 0xb7, 0xf3, 0x67, - 0x99, 0x8e, 0xbf, 0xd6, 0xd0, 0x5f, 0x6a, 0x42, 0x40, 0x2d, 0xa2, 0x84, 0x59, 0xf5, 0x26, 0x6a, - 0x2b, 0x54, 0x62, 0xc1, 0x5d, 0x44, 0x31, 0xb3, 0x94, 0x26, 0x6a, 0x9b, 0x54, 0x62, 0xc1, 0xc5, - 0x51, 0xca, 0x2c, 0xb5, 0xd4, 0x09, 0x4c, 0x2c, 0xd0, 0x13, 0x96, 0xe7, 0x41, 0xc8, 0x2c, 0x4d, - 0x4a, 0xab, 0x90, 0xec, 0x82, 0x19, 0xf2, 0x8c, 0x2f, 0x0b, 0x91, 0xd2, 0x90, 0x29, 0xbf, 0x09, - 0x79, 0x66, 0x10, 0xe6, 0x96, 0x5e, 0xd6, 0x17, 0x58, 0xd4, 0x9a, 0xf1, 0x65, 0x5a, 0xb0, 0xcc, - 0x32, 0x9a, 0xa8, 0xad, 0xd2, 0x2a, 0x24, 0x36, 0x40, 0xc6, 0xe6, 0xc1, 0xac, 0x08, 0xa6, 0x31, - 0xb3, 0xcc, 0x26, 0x6a, 0x1b, 0xf4, 0x01, 0x43, 0x9e, 0x81, 0x3e, 0xbb, 0x0c, 0xd2, 0x94, 0xc5, - 0x16, 0xc8, 0xd1, 0x77, 0x1e, 0x1b, 0xbd, 0x5f, 0x4a, 0x68, 0xa5, 0x6d, 0x7d, 0x42, 0xb0, 0x31, - 0x8c, 0x62, 0x36, 0x60, 0x45, 0x10, 0xc5, 0xb2, 0x81, 0x45, 0xc6, 0xc3, 0x2c, 0x48, 0xa4, 0x83, - 0x26, 0xad, 0x42, 0xd1, 0xee, 0x25, 0xcf, 0x0b, 0x69, 0x91, 0x49, 0x25, 0x26, 0x3b, 0x60, 0x2e, - 0x73, 0x96, 0x4d, 0xd2, 0x20, 0xa9, 0x7c, 0x32, 0x04, 0xe1, 0x05, 0xa5, 0x7f, 0xd2, 0x53, 0xed, - 0x81, 0xa7, 0xff, 0x80, 0xb2, 0x88, 0xe6, 0xa5, 0x17, 0x3d, 0x7d, 0x75, 0xb7, 0xa7, 0x9c, 0xb8, - 0x03, 0x2a, 0xb8, 0x97, 0xaa, 0xa1, 0x62, 0xad, 0xf5, 0x15, 0x81, 0x21, 0xfa, 0x71, 0xd3, 0x0b, - 0x2e, 0x2a, 0xc8, 0xca, 0x65, 0x27, 0x12, 0x93, 0xff, 0x00, 0xf2, 0xe8, 0x23, 0x9b, 0x4c, 0xaf, - 0x0a, 0x96, 0xaf, 0xf7, 0x65, 0x0a, 0xa6, 0x27, 0x08, 0xf2, 0x3f, 0x6c, 0x25, 0x7c, 0x3e, 0x11, - 0x87, 0x4d, 0xd2, 0x20, 0xe5, 0xb9, 0x6c, 0x4b, 0xa1, 0x9b, 0x09, 0x9f, 0x8f, 0xa3, 0x84, 0x79, - 0x82, 0x23, 0x2f, 0x40, 0x9f, 0x97, 0x03, 0xcb, 0x4d, 0x6e, 0x1c, 0xee, 0x3d, 0x66, 0xd6, 0x03, - 0x5f, 0x7a, 0xea, 0xf5, 0xdd, 0x5e, 0x8d, 0x56, 0x59, 0xfb, 0xef, 0xc0, 0xa8, 0xfe, 0x22, 0xb2, - 0x01, 0xfa, 0xa9, 0xf7, 0xca, 0x1b, 0x9d, 0x7b, 0xb8, 0x46, 0x0c, 0x50, 0x5d, 0x6f, 0x38, 0xc2, - 0x48, 0xd0, 0xe7, 0x5d, 0xea, 0xb9, 0xde, 0x11, 0xae, 0x13, 0x13, 0x34, 0x87, 0xd2, 0x11, 0xc5, - 0x8a, 0x80, 0xc3, 0xee, 0xb8, 0x7b, 0x8c, 0x55, 0x21, 0xf6, 0x46, 0x9e, 0x83, 0x35, 0x21, 0x1e, - 0x38, 0xc3, 0xee, 0xe9, 0xf1, 0x18, 0x37, 0xf6, 0x3f, 0x23, 0xd0, 0xd7, 0x8b, 0x22, 0x3a, 0x28, - 0x03, 0xe7, 0x0c, 0xd7, 0x84, 0xc2, 0x1f, 0x8f, 0x68, 0xf7, 0xc8, 0xc1, 0x0a, 0xd9, 0x04, 0xc3, - 0x77, 0x7c, 0xdf, 0x1d, 0x79, 0x3e, 0x56, 0xc9, 0x36, 0x60, 0xdf, 0xf1, 0x7c, 0x77, 0xec, 0x9e, - 0x39, 0x93, 0x6e, 0xbf, 0xef, 0xf8, 0x3e, 0xd6, 0xa5, 0xe6, 0xf5, 0xf1, 0xc4, 0x79, 0xe3, 0xf4, - 0xb1, 0x51, 0x45, 0x27, 0x0e, 0x1d, 0x62, 0x93, 0xfc, 0x0d, 0x7f, 0x89, 0xc8, 0xf5, 0xc6, 0x0e, - 0xf5, 0xba, 0x6b, 0x1a, 0x5a, 0xaa, 0x81, 0x30, 0x6a, 0xa9, 0x46, 0x1d, 0xd7, 0x5b, 0xaa, 0xa1, - 0x61, 0xad, 0xa5, 0x1a, 0x0d, 0xdc, 0xe8, 0x3d, 0xb9, 0xfe, 0x61, 0xd7, 0xae, 0x57, 0x36, 0xba, - 0x59, 0xd9, 0xe8, 0x76, 0x65, 0xa3, 0xef, 0x2b, 0x1b, 0x7d, 0xb9, 0xb7, 0x6b, 0x37, 0xf7, 0x76, - 0xed, 0xf6, 0xde, 0xae, 0xbd, 0xd5, 0xe4, 0xf5, 0x9d, 0x36, 0xe4, 0x8d, 0x7c, 0xfa, 0x33, 0x00, - 0x00, 0xff, 0xff, 0xba, 0xe5, 0x3d, 0xfa, 0xd7, 0x03, 0x00, 0x00, +var fileDescriptor_log_8a14f28d7f50d5e3 = []byte{ + // 660 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x6c, 0x53, 0xd1, 0x6a, 0xdb, 0x48, + 0x14, 0xb5, 0x2c, 0xc9, 0x92, 0x6e, 0x42, 0x98, 0x1d, 0xb2, 0xa0, 0xdd, 0x64, 0x15, 0x63, 0x16, + 0xd6, 0xe4, 0xc1, 0x81, 0x2c, 0x0b, 0xfb, 0xb6, 0xc8, 0xf6, 0x38, 0xd1, 0xae, 0x91, 0xb3, 0x23, + 0x27, 0x29, 0xed, 0x83, 0x91, 0xed, 0x89, 0x22, 0x2a, 0x6b, 0x8c, 0x24, 0x17, 0xd2, 0x7f, 0x28, + 0xf4, 0xad, 0x3f, 0xd0, 0xaf, 0xe8, 0x17, 0xe4, 0x31, 0x8f, 0x79, 0x0a, 0xad, 0xf3, 0x23, 0x65, + 0x46, 0x56, 0x1b, 0x68, 0x1e, 0x24, 0xce, 0x3d, 0x3a, 0xf7, 0xce, 0xbd, 0xe7, 0x6a, 0xc0, 0x5e, + 0x15, 0x71, 0x72, 0x94, 0xf0, 0x48, 0x3c, 0xcb, 0xa9, 0x78, 0x77, 0x96, 0x19, 0x2f, 0x38, 0xc6, + 0x33, 0x3e, 0x7b, 0x9d, 0xf1, 0x70, 0x76, 0xdd, 0x11, 0x9a, 0x4e, 0xc2, 0xa3, 0x5f, 0x77, 0x23, + 0x1e, 0x71, 0xf9, 0xf9, 0x48, 0xa0, 0x52, 0xd9, 0xfa, 0x54, 0x07, 0x9d, 0xa4, 0x45, 0x76, 0x83, + 0xff, 0x06, 0x33, 0x67, 0x6f, 0x58, 0x16, 0x17, 0x37, 0xb6, 0xd2, 0x54, 0xda, 0x3b, 0xc7, 0xfb, + 0x9d, 0x1f, 0xcb, 0x74, 0x82, 0x8d, 0x86, 0x7e, 0x53, 0x63, 0x0c, 0x5a, 0x11, 0x2f, 0x98, 0x5d, + 0x6f, 0x2a, 0x6d, 0x95, 0x4a, 0x2c, 0xb8, 0xab, 0x38, 0x61, 0xb6, 0xda, 0x54, 0xda, 0x16, 0x95, + 0x58, 0x70, 0x49, 0x9c, 0x32, 0x5b, 0x2b, 0x75, 0x02, 0x63, 0x1b, 0x8c, 0x05, 0xcb, 0xf3, 0x30, + 0x62, 0xb6, 0x2e, 0xa5, 0x55, 0x88, 0xf7, 0xc1, 0x8a, 0x78, 0xc6, 0x57, 0x85, 0x48, 0x69, 0xc8, + 0x94, 0xef, 0x84, 0x3c, 0x33, 0x8c, 0x72, 0xdb, 0x28, 0xeb, 0x0b, 0x2c, 0x6a, 0xcd, 0xf8, 0x2a, + 0x2d, 0x58, 0x66, 0x9b, 0x4d, 0xa5, 0xad, 0xd1, 0x2a, 0xc4, 0x0e, 0x40, 0xc6, 0xe6, 0xe1, 0xac, + 0x08, 0xa7, 0x09, 0xb3, 0xad, 0xa6, 0xd2, 0x36, 0xe9, 0x13, 0x06, 0xff, 0x05, 0xc6, 0xec, 0x3a, + 0x4c, 0x53, 0x96, 0xd8, 0x20, 0x47, 0xdf, 0x7b, 0x6e, 0xf4, 0x5e, 0x29, 0xa1, 0x95, 0xb6, 0xf5, + 0x4e, 0x81, 0xad, 0x41, 0x9c, 0xb0, 0x3e, 0x2b, 0xc2, 0x38, 0x91, 0x0d, 0x2c, 0x33, 0x1e, 0x65, + 0xe1, 0x42, 0x3a, 0x68, 0xd1, 0x2a, 0x14, 0xed, 0x5e, 0xf3, 0xbc, 0x90, 0x16, 0x59, 0x54, 0x62, + 0xbc, 0x07, 0xd6, 0x2a, 0x67, 0xd9, 0x24, 0x0d, 0x17, 0x95, 0x4f, 0xa6, 0x20, 0xfc, 0xb0, 0xf4, + 0x4f, 0x7a, 0xaa, 0x3f, 0xf1, 0xf4, 0x17, 0x50, 0x97, 0xf1, 0xbc, 0xf4, 0xa2, 0x6b, 0xac, 0x1f, + 0x0e, 0xd4, 0x33, 0xaf, 0x4f, 0x05, 0xf7, 0xaf, 0x66, 0x6a, 0x48, 0x6f, 0x7d, 0x54, 0xc0, 0x14, + 0xfd, 0x78, 0xe9, 0x15, 0x17, 0x15, 0x64, 0xe5, 0xb2, 0x13, 0x89, 0xf1, 0x6f, 0x00, 0x79, 0xfc, + 0x96, 0x4d, 0xa6, 0x37, 0x05, 0xcb, 0x37, 0xfb, 0xb2, 0x04, 0xd3, 0x15, 0x04, 0xfe, 0x1d, 0x76, + 0x16, 0x7c, 0x3e, 0x11, 0x87, 0x4d, 0xd2, 0x30, 0xe5, 0xb9, 0x6c, 0x4b, 0xa5, 0xdb, 0x0b, 0x3e, + 0x1f, 0xc7, 0x0b, 0xe6, 0x0b, 0x0e, 0xff, 0x03, 0xc6, 0xbc, 0x1c, 0x58, 0x6e, 0x72, 0xeb, 0xf8, + 0xe0, 0x39, 0xb3, 0x9e, 0xf8, 0xd2, 0xd5, 0x6e, 0x1f, 0x0e, 0x6a, 0xb4, 0xca, 0x3a, 0x7c, 0x05, + 0x66, 0xf5, 0x17, 0xe1, 0x2d, 0x30, 0xce, 0xfd, 0xff, 0xfc, 0xd1, 0xa5, 0x8f, 0x6a, 0xd8, 0x04, + 0xcd, 0xf3, 0x07, 0x23, 0xa4, 0x08, 0xfa, 0xd2, 0xa5, 0xbe, 0xe7, 0x9f, 0xa0, 0x3a, 0xb6, 0x40, + 0x27, 0x94, 0x8e, 0x28, 0x52, 0x05, 0x1c, 0xb8, 0x63, 0x77, 0x88, 0x34, 0x21, 0xf6, 0x47, 0x3e, + 0x41, 0xba, 0x10, 0xf7, 0xc9, 0xc0, 0x3d, 0x1f, 0x8e, 0x51, 0xe3, 0xf0, 0x83, 0x02, 0xc6, 0x66, + 0x51, 0xd8, 0x00, 0xb5, 0x4f, 0x2e, 0x50, 0x4d, 0x80, 0xd1, 0x59, 0x80, 0x14, 0x0c, 0xd0, 0x38, + 0x25, 0xee, 0x70, 0x7c, 0x8a, 0xea, 0x22, 0x2d, 0x18, 0x8f, 0xa8, 0x7b, 0x42, 0x90, 0x8a, 0xb7, + 0xc1, 0x0c, 0x48, 0x10, 0x78, 0x23, 0x3f, 0x40, 0x1a, 0xde, 0x05, 0x14, 0x10, 0x3f, 0xf0, 0xc6, + 0xde, 0x05, 0x99, 0xb8, 0xbd, 0x1e, 0x09, 0x02, 0x64, 0x48, 0xcd, 0xff, 0xc3, 0x09, 0x79, 0x41, + 0x7a, 0xc8, 0xac, 0xa2, 0x33, 0x42, 0x07, 0xc8, 0xc2, 0x3f, 0xc3, 0x4f, 0x22, 0xf2, 0xfc, 0x31, + 0xa1, 0xbe, 0xbb, 0xa1, 0xa1, 0xa5, 0x99, 0x3a, 0xd2, 0x5b, 0x9a, 0xd9, 0x40, 0x8d, 0xee, 0x1f, + 0xb7, 0x5f, 0x9c, 0xda, 0xed, 0xda, 0x51, 0xee, 0xd6, 0x8e, 0x72, 0xbf, 0x76, 0x94, 0xcf, 0x6b, + 0x47, 0x79, 0xff, 0xe8, 0xd4, 0xee, 0x1e, 0x9d, 0xda, 0xfd, 0xa3, 0x53, 0x7b, 0xa9, 0xcb, 0x7b, + 0x3c, 0x6d, 0xc8, 0xab, 0xf9, 0xe7, 0xd7, 0x00, 0x00, 0x00, 0xff, 0xff, 0x74, 0x5c, 0x12, 0x6b, + 0xe0, 0x03, 0x00, 0x00, } diff --git a/pkg/util/log/logpb/log.proto b/pkg/util/log/logpb/log.proto index 6e95299885ff..426f8f908f2d 100644 --- a/pkg/util/log/logpb/log.proto +++ b/pkg/util/log/logpb/log.proto @@ -66,8 +66,28 @@ enum Channel { // sensitive operational data. DEV = 0; - // TODO(knz): Add the OPS and HEALTH channels. - reserved 1,2; + // OPS is the channel used to report "point" operational events, + // initiated by user operators or automation: + // + // - operator or system actions on server processes: process starts, + // stops, shutdowns, crashes (if they can be logged), + // including each time: command-line parameters, current version being run. + // - actions that impact the topology of a cluster: node additions, + // removals, decommissions, etc. + // - job-related initiation or termination. + // - cluster setting changes. + // - zone configuration changes. + OPS = 1; + + // HEALTH is the channel used to report "background" operational + // events, initiated by CockroachDB or reporting on automatic processes: + // + // - current resource usage, including critical resource usage. + // - node-node connection events, including connection errors and + // gossip details. + // - range and table leasing events. + // - up-, down-replication; range unavailability. + HEALTH = 2; // STORAGE is the channel used to report low-level storage // layer events (RocksDB/Pebble). @@ -83,8 +103,8 @@ enum Channel { // numbering and synchronous writes. SESSIONS = 4; - // TODO(knz): Add the USER_ADMIN and PRIVILEGES channels. - reserved 5,6; + // TODO(knz): Define the USER_ADMIN and PRIVILEGES channels. + reserved 5, 6; // SENSITIVE_ACCESS is the channel used to report SQL // data access to sensitive data (when enabled):