Skip to content

Commit

Permalink
telemetry,sql: remove redaction from operational sql data
Browse files Browse the repository at this point in the history
Previously, when redaction was introduced into CRDB, all unidentifiable
strings were marked as redacted since that was the safer approach. We
expected to later return with a closer look and differentiate more
carefully between what should be redacted and what shouldn't.

SQL names have been identified as operational-sensitive data that should
not be redacted since it provides very useful debugging information and,
while user-controlled, do not typically contain user-data since those
would be stored in a Datum. This commit marks names as safe from
redaction for telemetry logging in cases where the
`sql.telemetry.query_sampling.enabled` cluster setting is enabled.

Additionally, some log tags such as client IP addresses are not to be
considered sensitive and are critical to debugging operational issues.
They have also been marked as safe.

In order to help with documenting these cases, a helper
`SafeOperational()` has been added to the `log` package. This helps us
mark strings as safe while documenting *why* we're doing so.

Resolves #76595

Release note (security update, ops change): When the
`sql.telemetry.query_sampling.enabled` cluster setting is enabled, SQL
names and client IPs are no longer redacted in telemetry logs.
  • Loading branch information
dhartunian committed Feb 18, 2022
1 parent 66dfdac commit 10add46
Show file tree
Hide file tree
Showing 13 changed files with 99 additions and 24 deletions.
2 changes: 1 addition & 1 deletion pkg/ccl/sqlproxyccl/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ func (s *Server) Serve(ctx context.Context, ln net.Listener) error {
s.metrics.CurConnCount.Inc(1)
defer s.metrics.CurConnCount.Dec(1)
remoteAddr := conn.RemoteAddr()
ctxWithTag := logtags.AddTag(ctx, "client", remoteAddr)
ctxWithTag := logtags.AddTag(ctx, "client", log.SafeOperational(remoteAddr))
if err := s.connHandler(ctxWithTag, conn); err != nil {
log.Infof(ctxWithTag, "connection error: %v", err)
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/authentication.go
Original file line number Diff line number Diff line change
Expand Up @@ -180,7 +180,7 @@ func (s *authenticationServer) UserLogin(
// It is only available for demo and test clusters.
func (s *authenticationServer) demoLogin(w http.ResponseWriter, req *http.Request) {
ctx := context.Background()
ctx = logtags.AddTag(ctx, "client", req.RemoteAddr)
ctx = logtags.AddTag(ctx, "client", log.SafeOperational(req.RemoteAddr))
ctx = logtags.AddTag(ctx, "demologin", nil)

fail := func(err error) {
Expand Down
2 changes: 1 addition & 1 deletion pkg/server/init_handshake.go
Original file line number Diff line number Diff line change
Expand Up @@ -520,7 +520,7 @@ func initHandshakeHelper(
go func(peerAddress string) {
defer handshaker.wg.Done()

peerCtx := logtags.AddTag(ctx, "peer", peerAddress)
peerCtx := logtags.AddTag(ctx, "peer", log.SafeOperational(peerAddress))
log.Ops.Infof(peerCtx, "starting handshake client for peer")
handshaker.runClient(peerCtx, peerAddress, addr.String())
}(peerAddress)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_executor_exec.go
Original file line number Diff line number Diff line change
Expand Up @@ -1218,7 +1218,7 @@ func (ex *connExecutor) handleTxnRowsGuardrails(
*alreadyLogged = shouldLog
}
if shouldLog {
commonSQLEventDetails := ex.planner.getCommonSQLEventDetails()
commonSQLEventDetails := ex.planner.getCommonSQLEventDetails(defaultRedactionOptions)
var event eventpb.EventPayload
if ex.executorType == executorTypeInternal {
if isRead {
Expand Down
30 changes: 27 additions & 3 deletions pkg/sql/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/settings"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/schemachanger/scexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
Expand Down Expand Up @@ -164,10 +165,33 @@ type eventLogOptions struct {
// If verboseTraceLevel is non-zero, its value is used as value for
// the vmodule filter. See exec_log for an example use.
verboseTraceLevel log.Level

// Additional redaction options, if necessary.
rOpts redactionOptions
}

// redactionOptions contains instructions on how to redact the SQL
// events.
type redactionOptions struct {
omitSQLNameRedaction bool
}

func (p *planner) getCommonSQLEventDetails() eventpb.CommonSQLEventDetails {
redactableStmt := formatStmtKeyAsRedactableString(p.extendedEvalCtx.VirtualSchemas, p.stmt.AST, p.extendedEvalCtx.EvalContext.Annotations)
func (ro *redactionOptions) toFlags() tree.FmtFlags {
if ro.omitSQLNameRedaction {
return tree.FmtOmitNameRedaction
}
return tree.FmtSimple
}

var defaultRedactionOptions = redactionOptions{
omitSQLNameRedaction: false,
}

func (p *planner) getCommonSQLEventDetails(opt redactionOptions) eventpb.CommonSQLEventDetails {
redactableStmt := formatStmtKeyAsRedactableString(
p.extendedEvalCtx.VirtualSchemas, p.stmt.AST,
p.extendedEvalCtx.EvalContext.Annotations, opt.toFlags(),
)
commonSQLEventDetails := eventpb.CommonSQLEventDetails{
Statement: redactableStmt,
Tag: p.stmt.AST.StatementTag(),
Expand Down Expand Up @@ -195,7 +219,7 @@ func (p *planner) logEventsWithOptions(
p.extendedEvalCtx.ExecCfg, p.txn,
1+depth,
opts,
p.getCommonSQLEventDetails(),
p.getCommonSQLEventDetails(opts.rOpts),
entries...)
}

Expand Down
16 changes: 15 additions & 1 deletion pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ func (p *planner) maybeLogStatementInternal(
}
if telemetryMetrics.maybeUpdateLastEmittedTime(telemetryMetrics.timeNow(), requiredTimeElapsed) {
skippedQueries := telemetryMetrics.resetSkippedQueryCount()
p.logEventsOnlyExternally(ctx, eventLogEntry{event: &eventpb.SampledQuery{
p.logOperationalEventsOnlyExternally(ctx, eventLogEntry{event: &eventpb.SampledQuery{
CommonSQLExecDetails: execDetails,
SkippedQueries: skippedQueries,
CostEstimate: p.curPlan.instrumentation.costEstimate,
Expand All @@ -405,6 +405,20 @@ func (p *planner) logEventsOnlyExternally(ctx context.Context, entries ...eventL
entries...)
}

// logOperationalEventsOnlyExternally is a helper that sets redaction
// options to omit SQL Name redaction. This is used when logging to
// the telemetry channel when we want additional metadata available.
func (p *planner) logOperationalEventsOnlyExternally(
ctx context.Context, entries ...eventLogEntry,
) {
// The API contract for logEventsWithOptions() is that it returns
// no error when system.eventlog is not written to.
_ = p.logEventsWithOptions(ctx,
2, /* depth: we want to use the caller location */
eventLogOptions{dst: LogExternally, rOpts: redactionOptions{omitSQLNameRedaction: true}},
entries...)
}

// maybeAudit marks the current plan being constructed as flagged
// for auditing if the table being touched has an auditing mode set.
// This is later picked up by maybeLogStatement() above.
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -3186,10 +3186,10 @@ func scrubStmtStatKey(vt VirtualTabler, key string) (string, bool) {
// formatStmtKeyAsRedactableString given an AST node this function will fully
// qualify names using annotations to format it out into a redactable string.
func formatStmtKeyAsRedactableString(
vt VirtualTabler, rootAST tree.Statement, ann *tree.Annotations,
vt VirtualTabler, rootAST tree.Statement, ann *tree.Annotations, fs tree.FmtFlags,
) redact.RedactableString {
f := tree.NewFmtCtx(
tree.FmtAlwaysQualifyTableNames|tree.FmtMarkRedactionNode,
tree.FmtAlwaysQualifyTableNames|tree.FmtMarkRedactionNode|fs,
tree.FmtAnnotations(ann),
tree.FmtReformatTableNames(hideNonVirtualTableNameFunc(vt)))
f.FormatNode(rootAST)
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/auth_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -697,7 +697,7 @@ func TestClientAddrOverride(t *testing.T) {
t.Log(e.Tags)
if strings.Contains(e.Tags, "client=") {
seenClient = true
if !strings.Contains(e.Tags, "client="+string(redact.StartMarker())+tc.specialAddr+":"+tc.specialPort+string(redact.EndMarker())) {
if !strings.Contains(e.Tags, "client="+tc.specialAddr+":"+tc.specialPort) {
t.Fatalf("expected override addr in log tags, got %+v", e)
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/pgwire/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -703,7 +703,7 @@ func (s *Server) ServeConn(ctx context.Context, conn net.Conn, socketType Socket
// Only now do we know the remote client address for sure (it may have
// been overridden by a status parameter).
connDetails.RemoteAddress = sArgs.RemoteAddr.String()
ctx = logtags.AddTag(ctx, "client", connDetails.RemoteAddress)
ctx = logtags.AddTag(ctx, "client", log.SafeOperational(connDetails.RemoteAddress))
sp.SetTag("client", attribute.StringValue(connDetails.RemoteAddress))

// If a test is hooking in some authentication option, load it.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/schema_change_plan_node.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ func (p *planner) FormatAstAsRedactableString(
) redact.RedactableString {
return formatStmtKeyAsRedactableString(p.getVirtualTabler(),
statement,
annotations)
annotations, tree.FmtSimple)
}

// SchemaChange provides the planNode for the new schema changer.
Expand Down
17 changes: 15 additions & 2 deletions pkg/sql/sem/tree/format.go
Original file line number Diff line number Diff line change
Expand Up @@ -141,7 +141,7 @@ const (
fmtFormatByteLiterals

// FmtMarkRedactionNode instructs the pretty printer to redact datums,
// constants, and simples names (i.e. Name, UnrestrictedName) from statements.
// constants, and simple names (i.e. Name, UnrestrictedName) from statements.
FmtMarkRedactionNode

// FmtSummary instructs the pretty printer to produced a summarized version
Expand All @@ -163,6 +163,11 @@ const (
// - Show columns up to 15 characters.
// - Show condition up to 15 characters.
FmtSummary

// FmtOmitNameRedaction instructs the pretty printer to omit redaction
// for simple names (i.e. Name, UnrestrictedName) from statements.
// This flag *overrides* `FmtMarkRedactionNode` above.
FmtOmitNameRedaction
)

// PasswordSubstitution is the string that replaces
Expand Down Expand Up @@ -651,7 +656,15 @@ func (ctx *FmtCtx) formatNodeMaybeMarkRedaction(n NodeFormatter) {
case *Placeholder:
// Placeholders should be printed as placeholder markers.
// Deliberately empty so we format as normal.
case Datum, Constant, *Name, *UnrestrictedName:
case *Name, *UnrestrictedName:
if ctx.flags.HasFlags(FmtOmitNameRedaction) {
break
}
ctx.WriteString(string(redact.StartMarker()))
v.Format(ctx)
ctx.WriteString(string(redact.EndMarker()))
return
case Datum, Constant:
ctx.WriteString(string(redact.StartMarker()))
v.Format(ctx)
ctx.WriteString(string(redact.EndMarker()))
Expand Down
32 changes: 23 additions & 9 deletions pkg/sql/telemetry_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

type stubTime struct {
Expand Down Expand Up @@ -105,12 +106,13 @@ func TestTelemetryLogging(t *testing.T) {
// - statement type DML, enough time has elapsed

testData := []struct {
name string
query string
execTimestampsSeconds []float64 // Execute the query with the following timestamps.
expectedLogStatement string
stubMaxEventFrequency int64
expectedSkipped []int // Expected skipped query count per expected log line.
name string
query string
execTimestampsSeconds []float64 // Execute the query with the following timestamps.
expectedLogStatement string
stubMaxEventFrequency int64
expectedSkipped []int // Expected skipped query count per expected log line.
expectedUnredactedTags []string
}{
{
// Test case with statement that is not of type DML.
Expand All @@ -123,16 +125,18 @@ func TestTelemetryLogging(t *testing.T) {
`TRUNCATE TABLE`,
1,
[]int{0, 0, 0, 0},
[]string{"client"},
},
{
// Test case with statement that is of type DML.
// The first statement should be logged.
"select-*-limit-1-query",
"SELECT * FROM t LIMIT 1;",
[]float64{3},
`SELECT * FROM \"\"›.‹\"\"›.‹t› LIMIT ‹1›`,
`SELECT * FROM \"\".\"\".t LIMIT ‹1›`,
1,
[]int{0},
[]string{"client"},
},
{
// Test case with statement that is of type DML.
Expand All @@ -141,19 +145,21 @@ func TestTelemetryLogging(t *testing.T) {
"select-*-limit-2-query",
"SELECT * FROM t LIMIT 2;",
[]float64{4, 4.1, 4.2, 5},
`SELECT * FROM \"\"›.‹\"\"›.‹t› LIMIT ‹2›`,
`SELECT * FROM \"\".\"\".t LIMIT ‹2›`,
1,
[]int{0, 2},
[]string{"client"},
},
{
// Test case with statement that is of type DML.
// Once required time has elapsed, the next statement should be logged.
"select-*-limit-3-query",
"SELECT * FROM t LIMIT 3;",
[]float64{6, 6.01, 6.05, 6.06, 6.1, 6.2},
`SELECT * FROM \"\"›.‹\"\"›.‹t› LIMIT ‹3›`,
`SELECT * FROM \"\".\"\".t LIMIT ‹3›`,
10,
[]int{0, 3, 0},
[]string{"client"},
},
}

Expand Down Expand Up @@ -220,6 +226,14 @@ func TestTelemetryLogging(t *testing.T) {
if !distRe.MatchString(e.Message) {
t.Errorf("expected to find Distribution but none was found")
}
for _, eTag := range tc.expectedUnredactedTags {
for _, tag := range strings.Split(e.Tags, ",") {
kv := strings.Split(tag, "=")
if kv[0] == eTag && strings.ContainsAny(kv[0], fmt.Sprintf("%s%s", redact.StartMarker(), redact.EndMarker())) {
t.Errorf("expected tag %s to be redacted within tags: %s", tag, e.Tags)
}
}
}
}
}
if logCount != expectedLogCount {
Expand Down
10 changes: 10 additions & 0 deletions pkg/util/log/redact.go
Original file line number Diff line number Diff line change
Expand Up @@ -205,3 +205,13 @@ func TestingSetRedactable(redactableLogs bool) (cleanup func()) {
}
}
}

// SafeOperational is a transparent wrapper around `redact.Safe` that
// acts as documentation for *why* the object is being marked as safe.
// In this case, the intent is to label this piece of information as
// "operational" data which is helpful for telemetry and operator
// actions. Typically, this includes schema structure and information
// about internals that is *not* user data or derived from user data.
func SafeOperational(s interface{}) redact.SafeValue {
return redact.Safe(s)
}

0 comments on commit 10add46

Please sign in to comment.