Skip to content

Commit

Permalink
cli: support COCKROACH_REDACTION_POLICY_MANAGED env var
Browse files Browse the repository at this point in the history
Currently, log redaction policies have no way to discern their own
runtime environment. Logged objects that may be considered sensitive
and unsafe in on-prem deployments of CockroachDB might be otherwise
safe when we're running within a managed service such as Cockroach
Cloud. For example, CLI argument lists included as part of the
`cockroach start` command are already known to those operating the
managed service, so there's no reason we should be redacting this
information from logs in this case.

This patch adds the `COCKROACH_REDACTION_POLICY_MANAGED` env var to
be handled in the various the start commands. This flag is plumbed
through to the global logging config object where the log package has
access to it.

We also introduce `log.SafeManaged(s interface{})`, which conditionally
marks an object with `redact.Safe()` depending on whether or not we
are running as a managed service. This is a successor to the original
`log.SafeOperational(s interface{})` function.

I believe that this new env var should not be advertised in
our public documentation, as its intended use is for those running
Cockroach Cloud.

This patch also implements a handful of these redaction tweaks.
While it's not meant to be exhaustive, it supplies a good
baseline example of how `log.SafeManaged()` is used, and
provides an immediate reduction of redaction when it comes
to critical information for support staff, such as CLI args,
network addresses, and more.

Additionally, a few log lines have been identified as being
redacted when it was not necessary. This patch also makes a
few spot improvements using `redact.Safe` as well in such
cases.

We intentionally avoid a release note here, as we aim to keep
the usage of this environment variable internal.

Release justification: low-risk, high benefit changes to existing
functionality. The new CLI flag has a minimal impact on DB
operations and provides high value reduction of log redaction,
which will be necessary for support staff with our latest compliance
requirements.

Release note: none
  • Loading branch information
abarganier committed Sep 7, 2022
1 parent 1ac25dc commit 4640c58
Show file tree
Hide file tree
Showing 18 changed files with 182 additions and 60 deletions.
50 changes: 26 additions & 24 deletions pkg/cli/start.go
Original file line number Diff line number Diff line change
Expand Up @@ -953,7 +953,7 @@ func reportServerInfo(
buf.Printf("CockroachDB %s starting at %s (took %0.1fs)\n", srvS, timeutil.Now(), timeutil.Since(startTime).Seconds())
buf.Printf("build:\t%s %s @ %s (%s)\n",
redact.Safe(info.Distribution), redact.Safe(info.Tag), redact.Safe(info.Time), redact.Safe(info.GoVersion))
buf.Printf("webui:\t%s\n", serverCfg.AdminURL())
buf.Printf("webui:\t%s\n", log.SafeManaged(serverCfg.AdminURL()))

// (Re-)compute the client connection URL. We cannot do this
// earlier (e.g. above, in the runStart function) because
Expand All @@ -964,53 +964,53 @@ func reportServerInfo(
log.Ops.Errorf(ctx, "failed computing the URL: %v", err)
return err
}
buf.Printf("sql:\t%s\n", pgURL.ToPQ())
buf.Printf("sql (JDBC):\t%s\n", pgURL.ToJDBC())
buf.Printf("sql:\t%s\n", log.SafeManaged(pgURL.ToPQ()))
buf.Printf("sql (JDBC):\t%s\n", log.SafeManaged(pgURL.ToJDBC()))

buf.Printf("RPC client flags:\t%s\n", clientFlagsRPC())
buf.Printf("RPC client flags:\t%s\n", log.SafeManaged(clientFlagsRPC()))
if len(serverCfg.SocketFile) != 0 {
buf.Printf("socket:\t%s\n", serverCfg.SocketFile)
buf.Printf("socket:\t%s\n", log.SafeManaged(serverCfg.SocketFile))
}
logNum := 1
_ = cliCtx.logConfig.IterateDirectories(func(d string) error {
if logNum == 1 {
// Backward-compatibility.
buf.Printf("logs:\t%s\n", d)
buf.Printf("logs:\t%s\n", log.SafeManaged(d))
} else {
buf.Printf("logs[%d]:\t%s\n", logNum, d)
buf.Printf("logs[%d]:\t%s\n", log.SafeManaged(logNum), log.SafeManaged(d))
}
logNum++
return nil
})
if serverCfg.Attrs != "" {
buf.Printf("attrs:\t%s\n", serverCfg.Attrs)
buf.Printf("attrs:\t%s\n", log.SafeManaged(serverCfg.Attrs))
}
if len(serverCfg.Locality.Tiers) > 0 {
buf.Printf("locality:\t%s\n", serverCfg.Locality)
buf.Printf("locality:\t%s\n", log.SafeManaged(serverCfg.Locality))
}
if tmpDir := serverCfg.SQLConfig.TempStorageConfig.Path; tmpDir != "" {
buf.Printf("temp dir:\t%s\n", tmpDir)
buf.Printf("temp dir:\t%s\n", log.SafeManaged(tmpDir))
}
if ext := st.ExternalIODir; ext != "" {
buf.Printf("external I/O path: \t%s\n", ext)
buf.Printf("external I/O path: \t%s\n", log.SafeManaged(ext))
} else {
buf.Printf("external I/O path: \t<disabled>\n")
}
for i, spec := range serverCfg.Stores.Specs {
buf.Printf("store[%d]:\t%s\n", i, spec)
buf.Printf("store[%d]:\t%s\n", i, log.SafeManaged(spec))
}
buf.Printf("storage engine: \t%s\n", &serverCfg.StorageEngine)

// Print the commong server identifiers.
if baseCfg.ClusterName != "" {
buf.Printf("cluster name:\t%s\n", baseCfg.ClusterName)
buf.Printf("cluster name:\t%s\n", log.SafeManaged(baseCfg.ClusterName))
}
clusterID := serverCfg.BaseConfig.ClusterIDContainer.Get()
if tenantClusterID.Equal(uuid.Nil) {
buf.Printf("clusterID:\t%s\n", clusterID)
buf.Printf("clusterID:\t%s\n", log.SafeManaged(clusterID))
} else {
buf.Printf("storage clusterID:\t%s\n", clusterID)
buf.Printf("tenant clusterID:\t%s\n", tenantClusterID)
buf.Printf("storage clusterID:\t%s\n", log.SafeManaged(clusterID))
buf.Printf("tenant clusterID:\t%s\n", log.SafeManaged(tenantClusterID))
}
nodeID := serverCfg.BaseConfig.IDContainer.Get()
if isHostNode {
Expand All @@ -1035,7 +1035,7 @@ func reportServerInfo(
buf.Printf("KV addresses:\t")
comma := redact.SafeString("")
for _, addr := range serverCfg.SQLConfig.TenantKVAddrs {
buf.Printf("%s%s", comma, addr)
buf.Printf("%s%s", comma, log.SafeManaged(addr))
comma = ", "
}
buf.Printf("\n")
Expand Down Expand Up @@ -1118,7 +1118,7 @@ func reportConfiguration(ctx context.Context) {
// 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.Ops.Infof(ctx, "process identity: %s", sysutil.ProcessIdentity())
log.Ops.Infof(ctx, "process identity: %s", log.SafeManaged(sysutil.ProcessIdentity()))
}

func maybeWarnMemorySizes(ctx context.Context) {
Expand All @@ -1133,7 +1133,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.Ops.Warningf(ctx, "%s", buf.String())
log.Ops.Warningf(ctx, "%s", redact.Safe(buf.String()))
}

// Check that the total suggested "max" memory is well below the available memory.
Expand Down Expand Up @@ -1216,13 +1216,13 @@ func setupAndInitializeLoggingAndProfiling(
"- Intruders with access to your machine or network can observe client-server traffic.\n"+
"- 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.SafeManaged(addr))
log.Ops.Shoutf(ctx, 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"+
"- %s",
build.MakeIssueURL(53404),
redact.Safe(build.MakeIssueURL(53404)),
redact.Safe(docs.URL("secure-a-cluster.html")),
)
}
Expand All @@ -1246,7 +1246,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.Ops.Infof(ctx, "%s", info.Short())
log.Ops.Infof(ctx, "%s", log.SafeManaged(info.Short()))

initTraceDir(ctx, serverCfg.InflightTraceDirName)
initCPUProfile(ctx, serverCfg.CPUProfileDirName, serverCfg.Settings)
Expand All @@ -1267,8 +1267,10 @@ func setupAndInitializeLoggingAndProfiling(
func initGEOS(ctx context.Context) {
loc, err := geos.EnsureInit(geos.EnsureInitErrorDisplayPrivate, startCtx.geoLibsDir)
if err != nil {
log.Ops.Warningf(ctx, "could not initialize GEOS - spatial functions may not be available: %v", err)
log.Ops.Warningf(ctx,
"could not initialize GEOS - spatial functions may not be available: %v",
log.SafeManaged(err))
} else {
log.Ops.Infof(ctx, "GEOS loaded from directory %s", loc)
log.Ops.Infof(ctx, "GEOS loaded from directory %s", log.SafeManaged(loc))
}
}
4 changes: 2 additions & 2 deletions pkg/server/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -537,7 +537,7 @@ func (cfg *Config) Report(ctx context.Context) {
} else {
log.Infof(ctx, "system total memory: %s", humanizeutil.IBytes(memSize))
}
log.Infof(ctx, "server configuration:\n%s", cfg)
log.Infof(ctx, "server configuration:\n%s", log.SafeManaged(cfg))
}

// Engines is a container of engines, allowing convenient closing.
Expand Down Expand Up @@ -731,7 +731,7 @@ func (cfg *Config) CreateEngines(ctx context.Context) (Engines, error) {
}

log.Infof(ctx, "%d storage engine%s initialized",
len(engines), util.Pluralize(int64(len(engines))))
len(engines), redact.Safe(util.Pluralize(int64(len(engines)))))
for _, s := range details {
log.Infof(ctx, "%v", s)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/goroutinedumper/goroutinedumper.go
Original file line number Diff line number Diff line change
Expand Up @@ -119,7 +119,7 @@ func NewGoroutineDumper(
return nil, errors.New("directory to store dumps could not be determined")
}

log.Infof(ctx, "writing goroutine dumps to %s", dir)
log.Infof(ctx, "writing goroutine dumps to %s", log.SafeManaged(dir))

gd := &GoroutineDumper{
heuristics: []heuristic{
Expand Down
6 changes: 3 additions & 3 deletions pkg/server/heapprofiler/activequeryprofiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,13 +76,13 @@ func NewActiveQueryProfiler(

maxMem, warn, err := memLimitFn()
if err != nil {
return nil, err
return nil, errors.Wrap(err, "failed to detect cgroup memory limit")
}
if warn != "" {
log.Warningf(ctx, "warning when reading cgroup memory limit: %s", warn)
log.Warningf(ctx, "warning when reading cgroup memory limit: %s", log.SafeManaged(warn))
}

log.Infof(ctx, "writing go query profiles to %s", dir)
log.Infof(ctx, "writing go query profiles to %s", log.SafeManaged(dir))
qp := &ActiveQueryProfiler{
profiler: profiler{
store: newProfileStore(dumpStore, QueryFileNamePrefix, QueryFileNameSuffix, st),
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/heapprofiler/activequeryprofiler_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,7 +36,7 @@ func TestNewActiveQueryProfiler(t *testing.T) {
{
name: "returns error when no access to cgroups",
wantErr: true,
errMsg: "cgroups not available",
errMsg: "failed to detect cgroup memory limit: cgroups not available",
storeDir: heapProfilerDirName,
limitFn: cgroupFnWithReturn(0, "", errors.New("cgroups not available")),
},
Expand Down
5 changes: 4 additions & 1 deletion pkg/server/heapprofiler/heapprofiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,10 @@ func NewHeapProfiler(ctx context.Context, dir string, st *cluster.Settings) (*He
return nil, errors.AssertionFailedf("need to specify dir for NewHeapProfiler")
}

log.Infof(ctx, "writing go heap profiles to %s at least every %s", dir, resetHighWaterMarkInterval)
log.Infof(ctx,
"writing go heap profiles to %s at least every %s",
log.SafeManaged(dir),
resetHighWaterMarkInterval)

dumpStore := dumpstore.NewStore(dir, maxCombinedFileSize, st)

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/heapprofiler/statsprofiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,7 +51,7 @@ func NewStatsProfiler(
return nil, errors.AssertionFailedf("need to specify dir for NewStatsProfiler")
}

log.Infof(ctx, "writing memory stats to %s at last every %s", dir, resetHighWaterMarkInterval)
log.Infof(ctx, "writing memory stats to %s at last every %s", log.SafeManaged(dir), resetHighWaterMarkInterval)

dumpStore := dumpstore.NewStore(dir, maxCombinedFileSize, st)

Expand Down
9 changes: 5 additions & 4 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -1518,14 +1518,15 @@ func (s *Server) PreStart(ctx context.Context) error {
logPendingLossOfQuorumRecoveryEvents(ctx, s.node.stores)

log.Ops.Infof(ctx, "starting %s server at %s (use: %s)",
redact.Safe(s.cfg.HTTPRequestScheme()), s.cfg.HTTPAddr, s.cfg.HTTPAdvertiseAddr)
redact.Safe(s.cfg.HTTPRequestScheme()), log.SafeManaged(s.cfg.HTTPAddr), log.SafeManaged(s.cfg.HTTPAdvertiseAddr))
rpcConnType := redact.SafeString("grpc/postgres")
if s.cfg.SplitListenSQL {
rpcConnType = "grpc"
log.Ops.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)",
log.SafeManaged(s.cfg.SQLAddr), log.SafeManaged(s.cfg.SQLAdvertiseAddr))
}
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.Ops.Infof(ctx, "starting %s server at %s", log.SafeManaged(rpcConnType), log.SafeManaged(s.cfg.Addr))
log.Ops.Infof(ctx, "advertising CockroachDB node at %s", log.SafeManaged(s.cfg.AdvertiseAddr))

log.Event(ctx, "accepting connections")

Expand Down
2 changes: 1 addition & 1 deletion pkg/server/tracedumper/tracedumper.go
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ func NewTraceDumper(ctx context.Context, dir string, st *cluster.Settings) *Trac
return nil
}

log.Infof(ctx, "writing job trace dumps to %s", dir)
log.Infof(ctx, "writing job trace dumps to %s", log.SafeManaged(dir))

td := &TraceDumper{
currentTime: timeutil.Now,
Expand Down
1 change: 1 addition & 0 deletions pkg/util/cgroups/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
"//pkg/util/log",
"//pkg/util/system",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)

Expand Down
8 changes: 6 additions & 2 deletions pkg/util/cgroups/cgroups.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/system"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

const (
Expand Down Expand Up @@ -427,7 +428,10 @@ func readInt64Value(
func detectCntrlPath(cgroupFilePath string, controller string) (string, error) {
cgroup, err := os.Open(cgroupFilePath)
if err != nil {
return "", errors.Wrapf(err, "failed to read %s cgroup from cgroups file: %s", controller, cgroupFilePath)
return "", errors.Wrapf(err,
"failed to read %s cgroup from cgroups file: %s",
redact.Safe(controller),
log.SafeManaged(cgroupFilePath))
}
defer func() { _ = cgroup.Close() }()

Expand Down Expand Up @@ -459,7 +463,7 @@ func detectCntrlPath(cgroupFilePath string, controller string) (string, error) {
func getCgroupDetails(mountinfoPath string, cRoot string, controller string) (string, int, error) {
info, err := os.Open(mountinfoPath)
if err != nil {
return "", 0, errors.Wrapf(err, "failed to read mounts info from file: %s", mountinfoPath)
return "", 0, errors.Wrapf(err, "failed to read mounts info from file: %s", log.SafeManaged(mountinfoPath))
}
defer func() {
_ = info.Close()
Expand Down
34 changes: 16 additions & 18 deletions pkg/util/envutil/env.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,9 @@ var safeVarRegistry = map[redact.SafeString]struct{}{
"GODEBUG": {},
"GOMAXPROCS": {},
"GOTRACEBACK": {},
// gRPC.
"GRPC_GO_LOG_SEVERITY_LEVEL": {},
"GRPC_GO_LOG_VERBOSITY_LEVEL": {},
}

// valueReportableUnsafeVarRegistry is the list of variables where we can
Expand All @@ -198,21 +201,19 @@ var safeVarRegistry = map[redact.SafeString]struct{}{
// that users would be unhappy to see them enclosed within redaction
// markers in log files.
var valueReportableUnsafeVarRegistry = map[redact.SafeString]struct{}{
"DEBUG_HTTP2_GOROUTINES": {},
"GRPC_GO_LOG_SEVERITY_LEVEL": {},
"GRPC_GO_LOG_VERBOSITY_LEVEL": {},
"HOST_IP": {},
"LANG": {},
"LC_ALL": {},
"LC_COLLATE": {},
"LC_CTYPE": {},
"LC_TIME": {},
"LC_NUMERIC": {},
"LC_MESSAGES": {},
"LS_METRICS_ENABLED": {},
"TERM": {},
"TZ": {},
"ZONEINFO": {},
"DEBUG_HTTP2_GOROUTINES": {},
"HOST_IP": {},
"LANG": {},
"LC_ALL": {},
"LC_COLLATE": {},
"LC_CTYPE": {},
"LC_TIME": {},
"LC_NUMERIC": {},
"LC_MESSAGES": {},
"LS_METRICS_ENABLED": {},
"TERM": {},
"TZ": {},
"ZONEINFO": {},
// From the Go runtime.
"LOCALDOMAIN": {},
"RES_OPTIONS": {},
Expand Down Expand Up @@ -258,9 +259,6 @@ var nameReportableUnsafeVarRegistry = map[redact.SafeString]struct{}{
"GAE_MODULE_NAME": {},
"GAE_PARTITION": {},
"GAE_SERVICE": {},
// gRPC.
"GRPC_GO_LOG_SEVERITY_LEVEL": {},
"GRPC_GO_LOG_VERBOSITY_LEVEL": {},
// Kerberos.
"KRB5CCNAME": {},
// Pprof.
Expand Down
1 change: 1 addition & 0 deletions pkg/util/log/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/util/caller",
"//pkg/util/ctxgroup",
"//pkg/util/envutil",
"//pkg/util/leaktest",
"//pkg/util/log/channel",
"//pkg/util/log/logconfig",
Expand Down
23 changes: 23 additions & 0 deletions pkg/util/log/clog.go
Original file line number Diff line number Diff line change
Expand Up @@ -99,6 +99,11 @@ type loggingT struct {
// to this logger already.
active bool
firstUseStack string

// redactionPolicyManaged indicates whether we're running as part of a managed
// service (sourced from COCKROACH_REDACTION_POLICY_MANAGED env var). Impacts
// log redaction policies for log args marked with SafeManaged.
redactionPolicyManaged bool
}

allSinkInfos sinkInfoRegistry
Expand Down Expand Up @@ -217,6 +222,24 @@ func (l *loggingT) signalFatalCh() {
}
}

// setManagedRedactionPolicy configures the logging setup to indicate if
// we are running as part of a managed service. see SafeManaged for details
// on how this impacts log redaction policies.
func (l *loggingT) setManagedRedactionPolicy(isManaged bool) {
l.mu.Lock()
defer l.mu.Unlock()
l.mu.redactionPolicyManaged = isManaged
}

// hasManagedRedactionPolicy indicates if the logging setup is being run
// as part of a managed service. see SafeManaged for details on how this
// impacts log redaction policies.
func (l *loggingT) hasManagedRedactionPolicy() bool {
l.mu.Lock()
defer l.mu.Unlock()
return l.mu.redactionPolicyManaged
}

// outputLogEntry marshals a log entry proto into bytes, and writes
// the data to the log files. If a trace location is set, stack traces
// are added to the entry before marshaling.
Expand Down
Loading

0 comments on commit 4640c58

Please sign in to comment.