Skip to content

Commit

Permalink
kvserver: redact tenant-bound trace recordings
Browse files Browse the repository at this point in the history
This commit changes the `(*Node).Batch` endpoint to redact
trace recordings that were collected on behalf of a tenant.

This notably does not affect the structured payloads. We
need at least one of them - ContentionEvent - to remain intact
as the keys within are used by SQL observability.

Touches cockroachdb#58610. It does not close it because there may be other
endpoints that support tracing, and even if there aren't, there
is nothing that keeps us from accidentally re-introducing this
problem again in the future.

Release note: None
  • Loading branch information
tbg committed Sep 22, 2021
1 parent 4226c57 commit 29342fa
Show file tree
Hide file tree
Showing 4 changed files with 163 additions and 6 deletions.
14 changes: 9 additions & 5 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -906,7 +906,7 @@ func checkNoUnknownRequest(reqs []roachpb.RequestUnion) *roachpb.UnsupportedRequ
}

func (n *Node) batchInternal(
ctx context.Context, args *roachpb.BatchRequest,
ctx context.Context, tenID roachpb.TenantID, args *roachpb.BatchRequest,
) (*roachpb.BatchResponse, error) {
if detail := checkNoUnknownRequest(args.Requests); detail != nil {
var br roachpb.BatchResponse
Expand All @@ -918,7 +918,7 @@ func (n *Node) batchInternal(
if err := n.stopper.RunTaskWithErr(ctx, "node.Node: batch", func(ctx context.Context) error {
var finishSpan func(*roachpb.BatchResponse)
// Shadow ctx from the outer function. Written like this to pass the linter.
ctx, finishSpan = n.setupSpanForIncomingRPC(ctx, grpcutil.IsLocalRequestContext(ctx))
ctx, finishSpan = n.setupSpanForIncomingRPC(ctx, tenID)
// NB: wrapped to delay br evaluation to its value when returning.
defer func() { finishSpan(br) }()
if log.HasSpanOrEvent(ctx) {
Expand Down Expand Up @@ -1022,7 +1022,7 @@ func (n *Node) Batch(
}
}
}
br, err := n.batchInternal(ctx, args)
br, err := n.batchInternal(ctx, tenantID, args)
if callAdmittedWorkDoneOnKVAdmissionQ {
n.kvAdmissionQ.AdmittedWorkDone(tenantID)
}
Expand Down Expand Up @@ -1061,14 +1061,14 @@ func (n *Node) Batch(
// in which the response is to serialized. The BatchResponse can
// be nil in case no response is to be returned to the rpc caller.
func (n *Node) setupSpanForIncomingRPC(
ctx context.Context, isLocalRequest bool,
ctx context.Context, tenID roachpb.TenantID,
) (context.Context, func(*roachpb.BatchResponse)) {
// The operation name matches the one created by the interceptor in the
// remoteTrace case below.
const opName = "/cockroach.roachpb.Internal/Batch"
tr := n.storeCfg.AmbientCtx.Tracer
var newSpan, grpcSpan *tracing.Span
if isLocalRequest {
if isLocalRequest := grpcutil.IsLocalRequestContext(ctx) && tenID == roachpb.SystemTenantID; isLocalRequest {
// This is a local request which circumvented gRPC. Start a span now.
ctx, newSpan = tracing.EnsureChildSpan(ctx, tr, opName)
// Set the same span.kind tag as the gRPC interceptor.
Expand Down Expand Up @@ -1096,7 +1096,11 @@ func (n *Node) setupSpanForIncomingRPC(
// If our local span descends from a parent on the other
// end of the RPC (i.e. the !isLocalRequest) case,
// attach the span recording to the batch response.
// Tenants get a redacted recording, i.e. with anything
// sensitive stripped out of the verbose messages. However,
// structured payloads stay untouched.
if rec := grpcSpan.GetRecording(); rec != nil {
maybeRedactRecording(tenID, rec)
br.CollectedSpans = append(br.CollectedSpans, rec...)
}
}
Expand Down
43 changes: 43 additions & 0 deletions pkg/server/node_tenant.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,43 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package server

import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/redact"
)

func maybeRedactRecording(tenID roachpb.TenantID, rec tracing.Recording) {
if tenID == roachpb.SystemTenantID {
return
}
// For tenants, strip the verbose log messages. See:
// https://github.com/cockroachdb/cockroach/issues/70407
for i := range rec {
sp := &rec[i]
sp.Tags = nil
for j := range sp.Logs {
record := &sp.Logs[j]
for k := range record.Fields {
field := &record.Fields[k]
if field.Key != tracingpb.LogMessageField {
// We don't have any of these fields, but let's not take any
// chances (our dependencies might slip them in).
field.Value = string(redact.RedactedMarker())
continue
}
field.Value = string(redact.RedactableString(field.Value).Redact())
}
}
}
}
110 changes: 110 additions & 0 deletions pkg/server/node_tenant_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,110 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package server

import (
"context"
"testing"
"time"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/tracing/tracingpb"
"github.com/cockroachdb/logtags"
ptypes "github.com/gogo/protobuf/types"
"github.com/stretchr/testify/require"
)

// TestMaybeRedactRecording verifies that maybeRedactRecording strips
// sensitive details for recordings consumed by tenants.
//
// See kvccl.TestTenantTracesAreRedacted for an end-to-end test of this.
func TestMaybeRedactRecording(t *testing.T) {
const (
msgNotSensitive = "msg-tenant-shown"
msgSensitive = "msg-tenant-hidden"
tagNotSensitive = "tag-tenant-shown"
tagSensitive = "tag-tenant-hidden"
)

mkRec := func() tracing.Recording {
t.Helper()
tags := (&logtags.Buffer{}).
Add("tag_sensitive", tagSensitive).
Add("tag_not_sensitive", log.Safe(tagNotSensitive))
ctx := logtags.WithTags(context.Background(), tags)
ctx, sp := tracing.NewTracer().StartSpanCtx(ctx, "foo", tracing.WithForceRealSpan())
sp.SetVerbose(true)

log.Eventf(ctx, "%s %s", msgSensitive, log.Safe(msgNotSensitive))
sp.SetTag("all_span_tags_are_stripped", "because_no_redactability")
sp.Finish()
rec := sp.GetRecording()
require.Len(t, rec, 1)
return rec
}

t.Run("regular-tenant", func(t *testing.T) {
rec := mkRec()
maybeRedactRecording(roachpb.MakeTenantID(100), rec)
require.Zero(t, rec[0].Tags)
require.Len(t, rec[0].Logs, 1)
msg := rec[0].Logs[0].Fields[0].Value
t.Log(msg)
require.NotContains(t, msg, msgSensitive)
require.NotContains(t, msg, tagSensitive)
require.Contains(t, msg, msgNotSensitive)
require.Contains(t, msg, tagNotSensitive)
})

t.Run("system-tenant", func(t *testing.T) {
rec := mkRec()
maybeRedactRecording(roachpb.SystemTenantID, rec)
require.Equal(t, map[string]string{
"_verbose": "1",
"all_span_tags_are_stripped": "because_no_redactability",
"tag_not_sensitive": tagNotSensitive,
"tag_sensitive": tagSensitive,
}, rec[0].Tags)
require.Len(t, rec[0].Logs, 1)
msg := rec[0].Logs[0].Fields[0].Value
t.Log(msg)
require.Contains(t, msg, msgSensitive)
require.Contains(t, msg, tagSensitive)
require.Contains(t, msg, msgNotSensitive)
require.Contains(t, msg, tagNotSensitive)
})

t.Run("no-unhandled-fields", func(t *testing.T) {
// Guard against a new sensitive field being added to RecordedSpan. If
// you're here to see why this test failed to compile, ensure that the
// change you're making to RecordedSpan does not include new sensitive data
// that may leak from the KV layer to tenants. If it does, update
// maybeRedactRecording appropriately.
type calcifiedRecordedSpan struct {
TraceID uint64
SpanID uint64
ParentSpanID uint64
Operation string
Baggage map[string]string
Tags map[string]string
StartTime time.Time
Duration time.Duration
Logs []tracingpb.LogRecord
DeprecatedInternalStructured []*ptypes.Any
GoroutineID uint64
Finished bool
StructuredRecords []tracingpb.StructuredRecord
}
_ = (*calcifiedRecordedSpan)((*tracingpb.RecordedSpan)(nil))
})
}
2 changes: 1 addition & 1 deletion pkg/server/node_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -638,7 +638,7 @@ func TestNodeSendUnknownBatchRequest(t *testing.T) {
Requests: make([]roachpb.RequestUnion, 1),
}
n := &Node{}
br, err := n.batchInternal(context.Background(), &ba)
br, err := n.batchInternal(context.Background(), roachpb.SystemTenantID, &ba)
if err != nil {
t.Fatal(err)
}
Expand Down

0 comments on commit 29342fa

Please sign in to comment.