Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
86475: cli: support `COCKROACH_REDACTION_POLICY_MANAGED` env var r=knz a=abarganier

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 `--managed` flag to 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 inspired by the original
`log.SafeOperational(s interface{})` function.

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

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 (cli change): `cockroach start` commands now have an
additional `--managed` flag that can be used to indicate whether
or not the node is running as part of a managed service (e.g.
Cockroach Cloud). Perhaps this shouldn't be advertised in our
public facing docs, as its only intended for use by those running
Cockroach Cloud and not for on-prem deployments.

Addresses #86316

86774: sql/schemachanger: version gate element creation r=Xiang-Gu a=ajwerner

Commit 1: fix minSupportedVersion of `ADD COLUMN` in new schema changer
from v22.1 to v22.2
Commit 2: We cannot create elements the old version of the code does not know about.

Release justification: fixed mixed version incompatibility
Release note: None

87317: sql: improve and clean up tracing a bit r=yuzefovich a=yuzefovich

**tracing: omit distsql ids from SHOW TRACE**

This commit removes the custom handling of tracing tags with
`cockroach.` prefix when populating the output of SHOW TRACE.
Previously, all tags with this prefix would be included into the "start
span" message, possibly taking up multiple lines in the SHOW TRACE
output. However, there is only one user of those tags - ids of different
components of DistSQL infrastructure, and I don't think it's helpful to
have those ids in the output at all, so this commit removes this ability
and makes the "start span" message nicer.

This special handling was introduced four years ago in
60978aa and at that time there might
have been a reason to have some special handling of these tags (so that
they become visible when viewing the jaeger trace), but that is not
necessary anymore (I believe because we now always propagate all tags
across nodes).

Release justification: low-risk cleanup.

Release note: None

**execinfra: clean up ProcessorBase a bit**

This commit performs the following cleanup:
- it removes the redundant `InternalClose` implementations. At some
point last year an "extended" version was introduced to take in
a closure to be called when the processor is being closed. There is only
one user for that, and it can itself do the necessary cleanup before
calling `InternalClose`
- it removes the update to `rowIdx` of `ProcOutputHelper` (which tracks
how many rows the helper has emitted) when the processor is closed. The
idea behind this was to protect from the future calls to `Next` method
so that the helper doesn't emit more rows once it is closed, but it is
not allowed by the interface anyway - once the processor is closed, no
new calls to `Next` are allowed, so this protection was meaningless.
However, what prompted me to look into this was the fact that the
`rowIdx` field was being set to `MaxInt64` which would trip up the stats
collection change in the following commit.

Release justification: low-risk cleanup.

Release note: None

**sql: improve tracing of some things**

This commit makes it so that we create a tracing span for all
processors. Previously, out of performance considerations, we elided the
spans for the columnarizer, materializer, planNodeToRowSource, and
flowCoordinator, but given the improvements to tracing in the last year
or so it doesn't seem necessary to do that anymore. In particular so
given that we don't create tracing spans by default any way, only when
the tracing is enabled for the statement.

Additionally, this commit adds a couple of tags to the tracing span of
the vectorized outbox (similar to what we have in the row-by-row
engine).

Release justification: low-risk improvement.

Release note: None

87468: clusterversion: require env var to do poison dev upgrades r=dt a=dt

Previously the offsetting of all in-development versions ensured that upgrading to one of these would mark the cluster as untrusted, dev-version-only, however the fact we did not offset already released versions meant that one could perform such an upgrade easily, by simply starting a dev binary in a stable release data directory, as upgrades happen by default automatically. This could lead to an inadvertent and irreversible conversion of a cluster to dev versions.

This changes the behavior to default to offsetting _all_ versions, not just the the new ones, which has the effect of also offset the version _from which_ a binary is willing to upgrade. This significantly reduces the risk of inadvertently upgrading a cluster to a dev version, as by default, the dev version will refuse to start in a release-version's data directory.

In some cases however it is useful to start a custom or development build in an existing data directory, e.g. a snapshot collected from production. For these cases, the env var COCKROACH_UPGRADE_TO_DEV_VERSION can be used to only offset the second defined version and above, meaning that the first version, which is typically the minBinaryVersion, is left alone, and that binary thus considers itself backwards compatible with that older release version and will thus be willing to start in / join that existing cluster.

Release note: none.

Release justification: bug fix in new functionality.

87474: ci: pass custom timeout to testrace in CI r=rickystewart a=healthy-pod

In #86363, we added a timeout to tests at the test binary level. Tests running with `--config=race` however use a custom timeout, different from the original default values set by bazel based on the test size.

This patch propagates those custom values to testrace in CI.

Release justification: Non-production code changes
Release note: None

Co-authored-by: Alex Barganier <[email protected]>
Co-authored-by: Xiang Gu <[email protected]>
Co-authored-by: Andrew Werner <[email protected]>
Co-authored-by: Yahor Yuzefovich <[email protected]>
Co-authored-by: David Taylor <[email protected]>
Co-authored-by: healthy-pod <[email protected]>
  • Loading branch information
7 people committed Sep 8, 2022
6 parents 3ff179c + 4640c58 + 7dbf545 + 7291e4d + 5630389 + 7ac059f commit ce55e1b
Show file tree
Hide file tree
Showing 57 changed files with 423 additions and 299 deletions.
2 changes: 2 additions & 0 deletions .bazelrc
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,8 @@ info --ui_event_filters=-WARNING

build:race --@io_bazel_rules_go//go/config:race "--test_env=GORACE=halt_on_error=1 log_path=stdout" --test_sharding_strategy=disabled
test:test --test_env=TZ=
# Note: these timeout values are used indirectly in `build/teamcity/cockroach/ci/tests/testrace_impl.sh`.
# If those values are updated, the script should be updated accordingly.
test:race --test_timeout=1200,6000,18000,72000

# CI should always run with `--config=ci` or `--config=cinolint`.
Expand Down
30 changes: 19 additions & 11 deletions build/teamcity/cockroach/ci/tests/testrace_impl.sh
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,7 @@ set -xeuo pipefail
# packages are expected to be formatted as go-style, e.g. ./pkg/cmd/bazci.

bazel build //pkg/cmd/bazci --config=ci
size_to_timeout=("small:1200" "medium:6000" "large:18000" "enormous:72000")
for pkg in "$@"
do
# Query to list all affected tests.
Expand All @@ -14,19 +15,26 @@ do
then
pkg="$pkg:all"
fi
tests=$(bazel query "kind(go_test, $pkg)" --output=label)

# Run affected tests.
for test in $tests
for kv in "${size_to_timeout[@]}";
do
if [[ ! -z $(bazel query "attr(tags, \"broken_in_bazel\", $test)") ]]
then
echo "Skipping test $test as it is broken in bazel"
continue
fi
$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci -- test --config=ci --config=race "$test" \
--test_env=COCKROACH_LOGIC_TESTS_SKIP=true \
--test_env=GOMAXPROCS=8
size="${kv%%:*}"
timeout="${kv#*:}"
go_timeout=$(($timeout - 5))
tests=$(bazel query "attr(size, $size, kind("go_test", tests($pkg)))" --output=label)
# Run affected tests.
for test in $tests
do
if [[ ! -z $(bazel query "attr(tags, \"broken_in_bazel\", $test)") ]]
then
echo "Skipping test $test as it is broken in bazel"
continue
fi
$(bazel info bazel-bin --config=ci)/pkg/cmd/bazci/bazci_/bazci -- test --config=ci --config=race "$test" \
--test_env=COCKROACH_LOGIC_TESTS_SKIP=true \
--test_env=GOMAXPROCS=8 \
--test_arg=-test.timeout="${go_timeout}s"
done
done
done

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))
}
}
1 change: 1 addition & 0 deletions pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ go_library(
deps = [
"//pkg/roachpb",
"//pkg/settings",
"//pkg/util/envutil",
"//pkg/util/log",
"//pkg/util/metric",
"//pkg/util/protoutil",
Expand Down
33 changes: 23 additions & 10 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,7 +10,10 @@

package clusterversion

import "github.com/cockroachdb/cockroach/pkg/roachpb"
import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/envutil"
)

// Key is a unique identifier for a version of CockroachDB.
type Key int
Expand Down Expand Up @@ -472,28 +475,38 @@ var rawVersionsSingleton = keyedVersions{
}

const (
// unstableVersionsAbove is a cluster version Key above which any upgrades in
// this version are considered unstable development-only versions if it is not
// negative, and upgrading to them should permanently move a cluster to
// development versions. On master it should be the minted version of the last
// release, while on release branches it can be set to invalidVersionKey to
// disable marking any versions as development versions.
unstableVersionsAbove = V22_1
// developmentBranch should be toggled to false on a release branch once the
// set of versions becomes append-only and associated upgrade implementations
// are frozen. It is always true on the main development branch.
developmentBranch = true

// finalVersion should be set on a release branch to the minted final cluster
// version key, e.g. to V22_2 on the release-22.2 branch once it is minted.
// Setting it has the effect of ensuring no versions are subsequently added.
finalVersion = invalidVersionKey
)

// devVersionsAbove is the version key above which all versions are offset to be
// development version when developmentBranch is true. By default this is all
// versions, by setting this to -1, but an env var can override this, to leave
// the first version un-offset. Doing so means that that version, which is
// generally minBinaryVersion as well, is unchanged, and thus allows upgrading a
// stable release data-dir to a dev version if desired.
var devVersionsAbove Key = func() Key {
if envutil.EnvOrDefaultBool("COCKROACH_UPGRADE_TO_DEV_VERSION", false) {
return invalidVersionKey + 1
}
return invalidVersionKey
}()

var versionsSingleton = func() keyedVersions {
if unstableVersionsAbove > invalidVersionKey {
if developmentBranch {
const devOffset = 1000000
// Throw every version above the last release (which will be none on a release
// branch) 1 million major versions into the future, so any "upgrade" to a
// release branch build will be a downgrade and thus blocked.
for i := range rawVersionsSingleton {
if rawVersionsSingleton[i].Key > unstableVersionsAbove {
if rawVersionsSingleton[i].Key > devVersionsAbove {
rawVersionsSingleton[i].Major += devOffset
}
}
Expand Down
1 change: 1 addition & 0 deletions pkg/cmd/roachtest/tests/versionupgrade.go
Original file line number Diff line number Diff line change
Expand Up @@ -431,6 +431,7 @@ func upgradeNodes(

binary := uploadVersion(ctx, t, c, c.Node(node), newVersion)
settings := install.MakeClusterSettings(install.BinaryOption(binary))
settings.Env = append(settings.Env, "COCKROACH_UPGRADE_TO_DEV_VERSION=true")
c.Start(ctx, t.L(), startOpts, settings, c.Node(node))
}
}
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/client_migration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -240,8 +240,8 @@ func TestMigrateWaitsForApplication(t *testing.T) {
blockApplicationCh := make(chan struct{})

// We're going to be migrating from startV to endV.
startV := roachpb.Version{Major: 41}
endV := roachpb.Version{Major: 42}
startV := roachpb.Version{Major: 1000041}
endV := roachpb.Version{Major: 1000042}

ctx := context.Background()
tc := testcluster.StartTestCluster(t, 3, base.TestClusterArgs{
Expand Down
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 @@ -732,7 +732,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
Loading

0 comments on commit ce55e1b

Please sign in to comment.