Skip to content

Commit

Permalink
Merge pull request #70759 from cockroachdb/blathers/backport-release-…
Browse files Browse the repository at this point in the history
…21.2-70562

release-21.2: log,kvserver: hand redacted KV traces to tenants
  • Loading branch information
erikgrinaker authored Sep 28, 2021
2 parents 218fbf5 + d49a5dd commit d5a0fe1
Show file tree
Hide file tree
Showing 27 changed files with 522 additions and 168 deletions.
3 changes: 3 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ go_test(
srcs = [
"connector_test.go",
"main_test.go",
"tenant_trace_test.go",
"tenant_upgrade_test.go",
],
embed = [":kvtenantccl"],
Expand All @@ -48,6 +49,8 @@ go_test(
"//pkg/gossip",
"//pkg/jobs",
"//pkg/kv/kvclient/kvtenant",
"//pkg/kv/kvserver",
"//pkg/kv/kvserver/kvserverbase",
"//pkg/migration",
"//pkg/migration/migrations",
"//pkg/roachpb:with-mocks",
Expand Down
108 changes: 108 additions & 0 deletions pkg/ccl/kvccl/kvtenantccl/tenant_trace_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,108 @@
// Copyright 2021 The Cockroach Authors.
//
// Licensed as a CockroachDB Enterprise file under the Cockroach Community
// License (the "License"); you may not use this file except in compliance with
// the License. You may obtain a copy of the License at
//
// https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt

package kvtenantccl_test

import (
"context"
gosql "database/sql"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver"
"github.com/cockroachdb/cockroach/pkg/kv/kvserver/kvserverbase"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/security"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/sqlutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

// TestTenantTracesAreRedacted is an end-to-end version of
// `kvserver.TestMaybeRedactRecording`.
func TestTenantTracesAreRedacted(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
ctx := context.Background()

const (
sensitiveString = "super-secret-stuff"
visibleString = "tenant-can-see-this"
)

getTrace := func(t *testing.T, db *gosql.DB) [][]string {
runner := sqlutils.MakeSQLRunner(db)
runner.Exec(t, `CREATE TABLE kv(k STRING PRIMARY KEY, v STRING)`)
runner.Exec(t, `
SET tracing = on;
INSERT INTO kv VALUES('k', 'v');
SELECT * FROM kv;
SET tracing = off;
`)
sl := runner.QueryStr(t, `SELECT * FROM [ SHOW TRACE FOR SESSION ]`)
t.Log(sqlutils.MatrixToStr(sl))
return sl
}

knobs := &kvserver.StoreTestingKnobs{}
knobs.EvalKnobs.TestingEvalFilter = func(args kvserverbase.FilterArgs) *roachpb.Error {
log.Eventf(args.Ctx, "%v", sensitiveString)
log.Eventf(args.Ctx, "%v", log.Safe(visibleString))
return nil
}
var args base.TestClusterArgs
args.ServerArgs.Knobs.Store = knobs
tc := serverutils.StartNewTestCluster(t, 1, args)
defer tc.Stopper().Stop(ctx)

t.Run("system-tenant", func(t *testing.T) {
db := tc.ServerConn(0)
defer db.Close()
results := getTrace(t, db)

var found bool
for _, sl := range results {
for _, s := range sl {
if strings.Contains(s, sensitiveString) {
found = true
}
}
}
require.True(t, found, "did not find '%q' in trace:\n%s",
sensitiveString, sqlutils.MatrixToStr(results),
)
})

t.Run("regular-tenant", func(t *testing.T) {
_, tenDB := serverutils.StartTenant(t, tc.Server(0), base.TestTenantArgs{
TenantID: roachpb.MakeTenantID(security.EmbeddedTenantIDs()[0]),
})
defer tenDB.Close()
results := getTrace(t, tenDB)

var found bool
for _, sl := range results {
for _, s := range sl {
if strings.Contains(s, sensitiveString) {
t.Fatalf(
"trace for tenant contained KV-level trace message '%q':\n%s",
sensitiveString, sqlutils.MatrixToStr(results),
)
}
if strings.Contains(s, visibleString) {
found = true
}
}
}
require.True(t, found, "trace for tenant missing trace message '%q':\n%s",
visibleString, sqlutils.MatrixToStr(results))
})
}
2 changes: 1 addition & 1 deletion pkg/kv/bulk/sst_batcher_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -227,7 +227,7 @@ func runTestImport(t *testing.T, batchSizeValue int64) {
for _, rec := range getRec() {
for _, l := range rec.Logs {
for _, line := range l.Fields {
if strings.Contains(line.Value, "SSTable cannot be added spanning range bounds") {
if strings.Contains(line.Value.StripMarkers(), "SSTable cannot be added spanning range bounds") {
splitRetries++
}
}
Expand Down
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/concurrency/concurrency_manager_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1024,7 +1024,7 @@ func (m *monitor) collectRecordings() string {
continue
}
logs = append(logs, logRecord{
g: g, value: field.Value,
g: g, value: field.Value.StripMarkers(),
})
g.prevEvents++
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/server/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_library(
"loopback.go",
"migration.go",
"node.go",
"node_tenant.go",
"node_tombstone_storage.go",
"pagination.go",
"problem_ranges.go",
Expand Down Expand Up @@ -281,6 +282,7 @@ go_test(
"main_test.go",
"migration_test.go",
"multi_store_test.go",
"node_tenant_test.go",
"node_test.go",
"node_tombstone_storage_test.go",
"pagination_test.go",
Expand Down Expand Up @@ -375,13 +377,16 @@ go_test(
"//pkg/util/stop",
"//pkg/util/syncutil",
"//pkg/util/timeutil",
"//pkg/util/tracing",
"//pkg/util/tracing/tracingpb",
"//pkg/util/uuid",
"@com_github_cockroachdb_datadriven//:datadriven",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_logtags//:logtags",
"@com_github_dustin_go_humanize//:go-humanize",
"@com_github_gogo_protobuf//jsonpb",
"@com_github_gogo_protobuf//proto",
"@com_github_gogo_protobuf//types",
"@com_github_grpc_ecosystem_grpc_gateway//runtime:go_default_library",
"@com_github_kr_pretty//:pretty",
"@com_github_lib_pq//:pq",
Expand Down
14 changes: 9 additions & 5 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -888,7 +888,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 @@ -900,7 +900,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 @@ -1004,7 +1004,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 @@ -1043,14 +1043,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 @@ -1078,7 +1078,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
54 changes: 54 additions & 0 deletions pkg/server/node_tenant.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,54 @@
// 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"
)

var sRedactedMarker = redact.RedactableString(redact.EscapeBytes(nil))

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 = sRedactedMarker
continue
}
if !sp.RedactableLogs {
// If we're handling a span that originated from an (early patch
// release) 22.1 node, all the containing information will be
// stripped. Note that this is not the common path here, as most
// information in the trace will be from the local node, which
// always creates redactable logs.
field.Value = sRedactedMarker
continue
}
field.Value = field.Value.Redact()
}
}
}
}
Loading

0 comments on commit d5a0fe1

Please sign in to comment.