Skip to content

Commit

Permalink
*: mark various non-sensitive strings as safe
Browse files Browse the repository at this point in the history
This change updates a few previously redacted constants and simple types
often printed to logs to be marked as safe, not needing redaction.
These do not represent sensitive data, such as such as the "s"
pluralizer, the type of Admission Control work queue, or a node's
liveness record, but they had not been marked safe or had
`SafeFormatter` implemented previously.

The types/constants marked as safe are listed as follows with examples:
- `livenesspb.Liveness`: e.g. `liveness(nid:124 epo:12 exp:12345.6789,0)`
- `livenesspb.MembershipStatus`: e.g. `active`, `decommissioning`
- `admission.WorkKind`: e.g. `kv`, `sql-kv-response`, etc
- `util.Pluralize()`: e.g. `s` on end of strings
- `upgrade.Name()`: e.g. `Upgrade to 0.0-2: "add users and roles"`

This includes a fix for the format string used in printing
`enginepb.EngineType` at startup, printing `pebble` instead of `‹2›`.

Lastly, this also adds `pebble.FormatMajorVersion` to the list of
allowed `redact.SafeValue`s in anticipation of
cockroachdb/pebble#2992, in the same vein as
this PR.

Epic: none

Release note: None
  • Loading branch information
AlexTalks authored and andrewbaptist committed Nov 14, 2023
1 parent 7ba9725 commit f9cd179
Show file tree
Hide file tree
Showing 12 changed files with 79 additions and 34 deletions.
1 change: 1 addition & 0 deletions pkg/kv/kvserver/liveness/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -75,6 +75,7 @@ go_test(
"//pkg/util/timeutil",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_cockroachdb_redact//:redact",
"@com_github_kr_pretty//:pretty",
"@com_github_stretchr_testify//assert",
"@com_github_stretchr_testify//require",
Expand Down
18 changes: 18 additions & 0 deletions pkg/kv/kvserver/liveness/liveness_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,9 +28,27 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/redact"
"github.com/stretchr/testify/require"
)

func TestLivenessRedaction(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

liveness := livenesspb.Liveness{
NodeID: roachpb.NodeID(13),
Epoch: 3,
Expiration: hlc.Timestamp{WallTime: 12345}.ToLegacyTimestamp(),
Draining: true,
Membership: livenesspb.MembershipStatus_ACTIVE,
}

require.EqualValues(t,
"liveness(nid:13 epo:3 exp:0.000012345,0 drain:true membership:active)",
redact.Sprintf("%+v", liveness).Redact())
}

func TestShouldReplaceLiveness(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
1 change: 1 addition & 0 deletions pkg/kv/kvserver/liveness/livenesspb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ go_library(
"//pkg/roachpb",
"//pkg/util/hlc",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
"@org_golang_google_grpc//codes",
"@org_golang_google_grpc//status",
],
Expand Down
15 changes: 12 additions & 3 deletions pkg/kv/kvserver/liveness/livenesspb/liveness.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
"google.golang.org/grpc/codes"
"google.golang.org/grpc/status"
)
Expand Down Expand Up @@ -67,11 +68,16 @@ func (l Liveness) Compare(o Liveness) int {
}

func (l Liveness) String() string {
var extra string
return redact.StringWithoutMarkers(l)
}

// SafeFormat implements the redact.SafeFormatter interface.
func (l Liveness) SafeFormat(s redact.SafePrinter, _ rune) {
s.Printf("liveness(nid:%d epo:%d exp:%s", l.NodeID, l.Epoch, l.Expiration)
if l.Draining || l.Membership.Decommissioning() || l.Membership.Decommissioned() {
extra = fmt.Sprintf(" drain:%t membership:%s", l.Draining, l.Membership.String())
s.Printf(" drain:%t membership:%s", l.Draining, l.Membership)
}
return fmt.Sprintf("liveness(nid:%d epo:%d exp:%s%s)", l.NodeID, l.Epoch, l.Expiration, extra)
s.Printf(")")
}

// Decommissioning is a shorthand to check if the membership status is DECOMMISSIONING.
Expand All @@ -83,6 +89,9 @@ func (c MembershipStatus) Decommissioned() bool { return c == MembershipStatus_D
// Active is a shorthand to check if the membership status is ACTIVE.
func (c MembershipStatus) Active() bool { return c == MembershipStatus_ACTIVE }

// SafeValue implements the redact.SafeValue interface.
func (MembershipStatus) SafeValue() {}

func (c MembershipStatus) String() string {
// NB: These strings must not be changed, since the CLI matches on them.
switch c {
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -753,7 +753,7 @@ func (n *Node) start(
allEngines = append(allEngines, state.uninitializedEngines...)
for _, e := range allEngines {
t := e.Type()
log.Infof(ctx, "started with engine type %v", t)
log.Infof(ctx, "started with engine type %v", &t)
}
log.Infof(ctx, "started with attributes %v", attrs.Attrs)
return nil
Expand Down
6 changes: 6 additions & 0 deletions pkg/testutils/lint/passes/redactcheck/redactcheck.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,6 +103,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) {
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverpb": {
"SnapshotRequest_Type": {},
},
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/liveness/livenesspb": {
"MembershipStatus": {},
},
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/spanset": {
"SpanAccess": {},
"SpanScope": {},
Expand Down Expand Up @@ -188,6 +191,9 @@ func runAnalyzer(pass *analysis.Pass) (interface{}, error) {
"TxnEpoch": {},
"TxnSeq": {},
},
"github.com/cockroachdb/cockroach/pkg/util/admission": {
"WorkKind": {},
},
"github.com/cockroachdb/cockroach/pkg/util/hlc": {
"ClockTimestamp": {},
"LegacyTimestamp": {},
Expand Down
4 changes: 2 additions & 2 deletions pkg/upgrade/upgrademanager/manager.go
Original file line number Diff line number Diff line change
Expand Up @@ -738,14 +738,14 @@ func (m *Manager) runMigration(
return err
}
if alreadyExisting {
log.Infof(ctx, "waiting for %s", mig.Name())
log.Infof(ctx, "waiting for %s", redact.Safe(mig.Name()))
return startup.RunIdempotentWithRetry(ctx,
m.deps.Stopper.ShouldQuiesce(),
"upgrade wait jobs", func(ctx context.Context) error {
return m.jr.WaitForJobs(ctx, []jobspb.JobID{id})
})
} else {
log.Infof(ctx, "running %s", mig.Name())
log.Infof(ctx, "running %s", redact.Safe(mig.Name()))
return startup.RunIdempotentWithRetry(ctx,
m.deps.Stopper.ShouldQuiesce(),
"upgrade run jobs", func(ctx context.Context) error {
Expand Down
8 changes: 6 additions & 2 deletions pkg/util/admission/admission.go
Original file line number Diff line number Diff line change
Expand Up @@ -526,8 +526,12 @@ const (
numWorkKinds
)

func workKindString(workKind WorkKind) string {
switch workKind {
// SafeValue implements the redact.SafeValue interface.
func (WorkKind) SafeValue() {}

// String implements the fmt.Stringer interface.
func (wk WorkKind) String() string {
switch wk {
case KVWork:
return "kv"
case SQLKVResponseWork:
Expand Down
33 changes: 17 additions & 16 deletions pkg/util/admission/grant_coordinator.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package admission

import (
"context"
"fmt"
"time"
"unsafe"

Expand Down Expand Up @@ -460,7 +461,7 @@ func makeStoresGrantCoordinators(
// the common priorities.
// TODO(baptist): Add per-store metrics.
storeWorkQueueMetrics :=
makeWorkQueueMetrics(workKindString(KVWork)+"-stores", registry,
makeWorkQueueMetrics(fmt.Sprintf("%s-stores", KVWork), registry,
admissionpb.TTLLowPri, admissionpb.BulkNormalPri,
admissionpb.NormalPri, admissionpb.LockingNormalPri)
makeStoreRequester := makeStoreWorkQueue
Expand Down Expand Up @@ -530,7 +531,7 @@ func makeRegularGrantCoordinator(
}

kvSlotAdjuster.granter = kvg
wqMetrics := makeWorkQueueMetrics(workKindString(KVWork), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
wqMetrics := makeWorkQueueMetrics(KVWork.String(), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
req := makeRequester(ambientCtx, KVWork, kvg, st, wqMetrics, makeWorkQueueOptions(KVWork))
coord.queues[KVWork] = req
kvg.requester = req
Expand All @@ -543,7 +544,7 @@ func makeRegularGrantCoordinator(
maxBurstTokens: opts.SQLKVResponseBurstTokens,
cpuOverload: kvSlotAdjuster,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLKVResponseWork), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
wqMetrics = makeWorkQueueMetrics(SQLKVResponseWork.String(), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
req = makeRequester(
ambientCtx, SQLKVResponseWork, tg, st, wqMetrics, makeWorkQueueOptions(SQLKVResponseWork))
coord.queues[SQLKVResponseWork] = req
Expand All @@ -557,7 +558,7 @@ func makeRegularGrantCoordinator(
maxBurstTokens: opts.SQLSQLResponseBurstTokens,
cpuOverload: kvSlotAdjuster,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLSQLResponseWork), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
wqMetrics = makeWorkQueueMetrics(SQLSQLResponseWork.String(), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
req = makeRequester(ambientCtx,
SQLSQLResponseWork, tg, st, wqMetrics, makeWorkQueueOptions(SQLSQLResponseWork))
coord.queues[SQLSQLResponseWork] = req
Expand All @@ -571,7 +572,7 @@ func makeRegularGrantCoordinator(
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
wqMetrics = makeWorkQueueMetrics(SQLStatementLeafStartWork.String(), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
req = makeRequester(ambientCtx,
SQLStatementLeafStartWork, sg, st, wqMetrics, makeWorkQueueOptions(SQLStatementLeafStartWork))
coord.queues[SQLStatementLeafStartWork] = req
Expand All @@ -585,7 +586,7 @@ func makeRegularGrantCoordinator(
cpuOverload: kvSlotAdjuster,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
wqMetrics = makeWorkQueueMetrics(SQLStatementRootStartWork.String(), registry, admissionpb.NormalPri, admissionpb.LockingNormalPri)
req = makeRequester(ambientCtx,
SQLStatementRootStartWork, sg, st, wqMetrics, makeWorkQueueOptions(SQLStatementRootStartWork))
coord.queues[SQLStatementRootStartWork] = req
Expand Down Expand Up @@ -628,7 +629,7 @@ func NewGrantCoordinatorSQL(
maxBurstTokens: opts.SQLKVResponseBurstTokens,
cpuOverload: sqlNodeCPU,
}
wqMetrics := makeWorkQueueMetrics(workKindString(SQLKVResponseWork), registry)
wqMetrics := makeWorkQueueMetrics(SQLKVResponseWork.String(), registry)
req := makeRequester(ambientCtx,
SQLKVResponseWork, tg, st, wqMetrics, makeWorkQueueOptions(SQLKVResponseWork))
coord.queues[SQLKVResponseWork] = req
Expand All @@ -642,7 +643,7 @@ func NewGrantCoordinatorSQL(
maxBurstTokens: opts.SQLSQLResponseBurstTokens,
cpuOverload: sqlNodeCPU,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLSQLResponseWork), registry)
wqMetrics = makeWorkQueueMetrics(SQLSQLResponseWork.String(), registry)
req = makeRequester(ambientCtx,
SQLSQLResponseWork, tg, st, wqMetrics, makeWorkQueueOptions(SQLSQLResponseWork))
coord.queues[SQLSQLResponseWork] = req
Expand All @@ -656,7 +657,7 @@ func NewGrantCoordinatorSQL(
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLLeafStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementLeafStartWork), registry)
wqMetrics = makeWorkQueueMetrics(SQLStatementLeafStartWork.String(), registry)
req = makeRequester(ambientCtx,
SQLStatementLeafStartWork, sg, st, wqMetrics, makeWorkQueueOptions(SQLStatementLeafStartWork))
coord.queues[SQLStatementLeafStartWork] = req
Expand All @@ -670,7 +671,7 @@ func NewGrantCoordinatorSQL(
cpuOverload: sqlNodeCPU,
usedSlotsMetric: metrics.SQLRootStartUsedSlots,
}
wqMetrics = makeWorkQueueMetrics(workKindString(SQLStatementRootStartWork), registry)
wqMetrics = makeWorkQueueMetrics(SQLStatementRootStartWork.String(), registry)
req = makeRequester(ambientCtx,
SQLStatementRootStartWork, sg, st, wqMetrics, makeWorkQueueOptions(SQLStatementRootStartWork))
coord.queues[SQLStatementRootStartWork] = req
Expand Down Expand Up @@ -986,7 +987,7 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, _ rune) {
case KVWork:
switch g := coord.granters[i].(type) {
case *slotGranter:
s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots)
s.Printf("%s%s: used: %d, total: %d", curSep, kind, g.usedSlots, g.totalSlots)
case *kvStoreTokenGranter:
s.Printf(" io-avail: %d(%d), elastic-disk-bw-tokens-avail: %d", g.coordMu.availableIOTokens,
g.coordMu.availableElasticIOTokens,
Expand All @@ -995,12 +996,12 @@ func (coord *GrantCoordinator) SafeFormat(s redact.SafePrinter, _ rune) {
case SQLStatementLeafStartWork, SQLStatementRootStartWork:
if coord.granters[i] != nil {
g := coord.granters[i].(*slotGranter)
s.Printf("%s%s: used: %d, total: %d", curSep, workKindString(kind), g.usedSlots, g.totalSlots)
s.Printf("%s%s: used: %d, total: %d", curSep, kind, g.usedSlots, g.totalSlots)
}
case SQLKVResponseWork, SQLSQLResponseWork:
if coord.granters[i] != nil {
g := coord.granters[i].(*tokenGranter)
s.Printf("%s%s: avail: %d", curSep, workKindString(kind), g.availableBurstTokens)
s.Printf("%s%s: avail: %d", curSep, kind, g.availableBurstTokens)
if kind == SQLKVResponseWork {
curSep = newlineStr
} else {
Expand Down Expand Up @@ -1039,15 +1040,15 @@ func (GrantCoordinatorMetrics) MetricStruct() {}
func makeGrantCoordinatorMetrics() GrantCoordinatorMetrics {
m := GrantCoordinatorMetrics{
KVTotalSlots: metric.NewGauge(totalSlots),
KVUsedSlots: metric.NewGauge(addName(workKindString(KVWork), usedSlots)),
KVUsedSlots: metric.NewGauge(addName(KVWork.String(), usedSlots)),
KVSlotsExhaustedDuration: metric.NewCounter(kvSlotsExhaustedDuration),
KVCPULoadShortPeriodDuration: metric.NewCounter(kvCPULoadShortPeriodDuration),
KVCPULoadLongPeriodDuration: metric.NewCounter(kvCPULoadLongPeriodDuration),
KVSlotAdjusterIncrements: metric.NewCounter(kvSlotAdjusterIncrements),
KVSlotAdjusterDecrements: metric.NewCounter(kvSlotAdjusterDecrements),
KVIOTokensExhaustedDuration: metric.NewCounter(kvIOTokensExhaustedDuration),
SQLLeafStartUsedSlots: metric.NewGauge(addName(workKindString(SQLStatementLeafStartWork), usedSlots)),
SQLRootStartUsedSlots: metric.NewGauge(addName(workKindString(SQLStatementRootStartWork), usedSlots)),
SQLLeafStartUsedSlots: metric.NewGauge(addName(SQLStatementLeafStartWork.String(), usedSlots)),
SQLRootStartUsedSlots: metric.NewGauge(addName(SQLStatementRootStartWork.String(), usedSlots)),
KVIOTokensTaken: metric.NewCounter(kvIOTokensTaken),
KVIOTokensReturned: metric.NewCounter(kvIOTokensReturned),
KVIOTokensBypassed: metric.NewCounter(kvIOTokensBypassed),
Expand Down
10 changes: 5 additions & 5 deletions pkg/util/admission/granter_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -398,7 +398,7 @@ func (tr *testRequester) hasWaitingRequests() bool {

func (tr *testRequester) granted(grantChainID grantChainID) int64 {
fmt.Fprintf(tr.buf, "%s%s: granted in chain %d, and returning %d\n",
workKindString(tr.workKind), tr.additionalID,
tr.workKind, tr.additionalID,
grantChainID, tr.returnValueFromGranted)
tr.grantChainID = grantChainID
return tr.returnValueFromGranted
Expand All @@ -408,24 +408,24 @@ func (tr *testRequester) close() {}

func (tr *testRequester) tryGet(count int64) {
rv := tr.granter.tryGet(count)
fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", workKindString(tr.workKind),
fmt.Fprintf(tr.buf, "%s%s: tryGet(%d) returned %t\n", tr.workKind,
tr.additionalID, count, rv)
}

func (tr *testRequester) returnGrant(count int64) {
fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", workKindString(tr.workKind), tr.additionalID,
fmt.Fprintf(tr.buf, "%s%s: returnGrant(%d)\n", tr.workKind, tr.additionalID,
count)
tr.granter.returnGrant(count)
}

func (tr *testRequester) tookWithoutPermission(count int64) {
fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", workKindString(tr.workKind),
fmt.Fprintf(tr.buf, "%s%s: tookWithoutPermission(%d)\n", tr.workKind,
tr.additionalID, count)
tr.granter.tookWithoutPermission(count)
}

func (tr *testRequester) continueGrantChain() {
fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", workKindString(tr.workKind),
fmt.Fprintf(tr.buf, "%s%s: continueGrantChain\n", tr.workKind,
tr.additionalID)
tr.granter.continueGrantChain(tr.grantChainID)
}
Expand Down
11 changes: 7 additions & 4 deletions pkg/util/admission/work_queue.go
Original file line number Diff line number Diff line change
Expand Up @@ -539,7 +539,7 @@ func (q *WorkQueue) tryCloseEpoch(timeNow time.Time) {
// specifically all the store WorkQueues share the same metric. We
// should eliminate that sharing and make those per store metrics.
log.Infof(q.ambientCtx, "%s: FIFO threshold for tenant %d %s %d",
workKindString(q.workKind), tenant.id, logVerb, tenant.fifoPriorityThreshold)
q.workKind, tenant.id, logVerb, tenant.fifoPriorityThreshold)
}
// Note that we are ignoring the new priority threshold and only
// dequeueing the ones that are in the closed epoch. It is possible to
Expand Down Expand Up @@ -718,7 +718,7 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err
deadline, _ := ctx.Deadline()
return true,
errors.Newf("work %s deadline already expired: deadline: %v, now: %v",
workKindString(q.workKind), deadline, startTime)
q.workKind, deadline, startTime)
}
// Push onto heap(s).
ordering := fifoWorkOrdering
Expand Down Expand Up @@ -804,9 +804,11 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err
q.metrics.recordFinishWait(info.Priority, waitDur)
deadline, _ := ctx.Deadline()
recordAdmissionWorkQueueStats(span, waitDur, q.workKind, true)
log.Eventf(ctx, "deadline expired, waited in %s queue for %v",
q.workKind, waitDur)
return true,
errors.Newf("work %s deadline expired while waiting: deadline: %v, start: %v, dur: %v",
workKindString(q.workKind), deadline, startTime, waitDur)
q.workKind, deadline, startTime, waitDur)
case chainID, ok := <-work.ch:
if !ok {
panic(errors.AssertionFailedf("channel should not be closed"))
Expand All @@ -818,6 +820,7 @@ func (q *WorkQueue) Admit(ctx context.Context, info WorkInfo) (enabled bool, err
panic(errors.AssertionFailedf("grantee should be removed from heap"))
}
recordAdmissionWorkQueueStats(span, waitDur, q.workKind, false)
log.Eventf(ctx, "admitted, waited in %s queue for %v", q.workKind, waitDur)
q.granter.continueGrantChain(chainID)
return true, nil
}
Expand All @@ -831,7 +834,7 @@ func recordAdmissionWorkQueueStats(
}
span.RecordStructured(&admissionpb.AdmissionWorkQueueStats{
WaitDurationNanos: waitDur,
WorkKind: workKindString(workKind),
WorkKind: workKind.String(),
DeadlineExceeded: deadlineExceeded,
})
}
Expand Down
4 changes: 3 additions & 1 deletion pkg/util/pluralize.go
Original file line number Diff line number Diff line change
Expand Up @@ -10,8 +10,10 @@

package util

import "github.com/cockroachdb/redact"

// Pluralize returns a single character 's' unless n == 1.
func Pluralize(n int64) string {
func Pluralize(n int64) redact.SafeString {
if n == 1 {
return ""
}
Expand Down

0 comments on commit f9cd179

Please sign in to comment.