Skip to content

Commit

Permalink
*: new logging channels OPS and HEALTH
Browse files Browse the repository at this point in the history
Release note (cli change): Logging events that are relevant to cluster
operators are now categorized under the new OPS and HEALTH logging
channels. These can now be redirected separately from other logging
events.

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 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.
  • Loading branch information
knz committed Nov 30, 2020
1 parent a6255d5 commit de2e1a9
Show file tree
Hide file tree
Showing 28 changed files with 1,119 additions and 185 deletions.
25 changes: 25 additions & 0 deletions docs/generated/logging.md
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -380,7 +380,7 @@ func maybeShoutError(
}

func checkAndMaybeShout(err error) error {
return checkAndMaybeShoutTo(err, log.Shoutf)
return checkAndMaybeShoutTo(err, log.Ops.Shoutf)
}

func checkAndMaybeShoutTo(
Expand Down
1 change: 1 addition & 0 deletions pkg/cli/flags_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -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,
Expand Down
4 changes: 2 additions & 2 deletions pkg/cli/initial_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
}

Expand Down
8 changes: 3 additions & 5 deletions pkg/cli/log_flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -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.")
}
Expand Down Expand Up @@ -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,
}
61 changes: 30 additions & 31 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
}

Expand All @@ -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.
Expand All @@ -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. ")
Expand All @@ -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)
}
}

Expand All @@ -486,26 +486,26 @@ 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)
}
}

// Set up the Geospatial library.
// 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
Expand Down Expand Up @@ -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)
Expand Down Expand Up @@ -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())
}
Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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 {
Expand Down Expand Up @@ -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.
Expand All @@ -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:
Expand Down Expand Up @@ -853,26 +852,26 @@ 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)
panic("unreachable")

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
Expand Down Expand Up @@ -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"+
Expand All @@ -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) {
Expand All @@ -952,15 +951,15 @@ 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.
if maxMemory, err := status.GetTotalMemory(ctx); err == nil {
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))
}
Expand All @@ -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"+
Expand All @@ -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"+
Expand All @@ -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()
Expand Down
Loading

0 comments on commit de2e1a9

Please sign in to comment.