Skip to content

Commit

Permalink
sql: implement fine-grained transaction error redaction
Browse files Browse the repository at this point in the history
Previously, transaction errors would be redacted in full since there was no
redaction implemented for their internals. This commit updates all the errors
defined in pkg/roachpb/errors.proto to implement `SafeErrorFormatter` in order
to be properly redacted.

The methods `message` in `ErrorDetailInterface` contained no usages so it was
removed.

The tests in place are checking to see that redaction markers *do not* appear
in the logs. They will be in place when Errors contain redacted values such as
Keys containing row data.

An additional test was added to the telemetry logging suite to ensure that
these errors are surfaced unredacted in telemetry logs as well.

Some message fields in Error protobufs have been updated to use
`RedactedString` to better support fine grained redaction. A simple migration
was applied for those fields but further work may be necessary for further
improvements.

Epic: CRDB-12732
Resolves: CRDB-14087

Release note (ops change): Transaction errors will contain more detailed
information in redacted logs.
  • Loading branch information
dhartunian committed Jan 13, 2023
1 parent c3e80e0 commit 3e1cdb0
Show file tree
Hide file tree
Showing 23 changed files with 554 additions and 213 deletions.
1 change: 1 addition & 0 deletions pkg/kv/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ go_library(
"//pkg/util/uuid",
"@com_github_cockroachdb_apd_v3//:apd",
"@com_github_cockroachdb_errors//:errors",
"@com_github_cockroachdb_redact//:redact",
],
)

Expand Down
12 changes: 7 additions & 5 deletions pkg/kv/kvclient/kvcoord/txn_coord_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package kvcoord

import (
"context"
"fmt"
"runtime/debug"

"github.com/cockroachdb/cockroach/pkg/kv"
Expand All @@ -25,6 +24,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/logtags"
"github.com/cockroachdb/redact"
"go.opentelemetry.io/otel/attribute"
)

Expand Down Expand Up @@ -424,7 +424,7 @@ func generateTxnDeadlineExceededErr(
txn *roachpb.Transaction, deadline hlc.Timestamp,
) *roachpb.Error {
exceededBy := txn.WriteTimestamp.GoTime().Sub(deadline.GoTime())
extraMsg := fmt.Sprintf(
extraMsg := redact.Sprintf(
"txn timestamp pushed too much; deadline exceeded by %s (%s > %s)",
exceededBy, txn.WriteTimestamp, deadline)
return roachpb.NewErrorWithTxn(
Expand Down Expand Up @@ -699,7 +699,7 @@ func (tc *TxnCoordSender) maybeRejectClientLocked(
case txnError:
return tc.mu.storedErr
case txnFinalized:
msg := fmt.Sprintf("client already committed or rolled back the transaction. "+
msg := redact.Sprintf("client already committed or rolled back the transaction. "+
"Trying to execute: %s", ba.Summary())
stack := string(debug.Stack())
log.Errorf(ctx, "%s. stack:\n%s", msg, stack)
Expand Down Expand Up @@ -813,7 +813,7 @@ func (tc *TxnCoordSender) handleRetryableErrLocked(

// We'll pass a TransactionRetryWithProtoRefreshError up to the next layer.
retErr := roachpb.NewTransactionRetryWithProtoRefreshError(
pErr.String(),
redact.Sprint(pErr),
errTxnID, // the id of the transaction that encountered the error
newTxn)

Expand Down Expand Up @@ -1270,7 +1270,9 @@ func (tc *TxnCoordSender) TestingCloneTxn() *roachpb.Transaction {
}

// PrepareRetryableError is part of the client.TxnSender interface.
func (tc *TxnCoordSender) PrepareRetryableError(ctx context.Context, msg string) error {
func (tc *TxnCoordSender) PrepareRetryableError(
ctx context.Context, msg redact.RedactableString,
) error {
tc.mu.Lock()
defer tc.mu.Unlock()
if tc.mu.txnState != txnPending {
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_pipeliner.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package kvcoord

import (
"context"
"fmt"
"sort"

"github.com/cockroachdb/cockroach/pkg/kv/kvserver/concurrency/lock"
Expand All @@ -23,6 +22,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/redact"
"github.com/google/btree"
)

Expand Down Expand Up @@ -778,7 +778,7 @@ func (tp *txnPipeliner) adjustError(
if ime, ok := pErr.GetDetail().(*roachpb.IntentMissingError); ok {
log.VEventf(ctx, 2, "transforming intent missing error into retry: %v", ime)
err := roachpb.NewTransactionRetryError(
roachpb.RETRY_ASYNC_WRITE_FAILURE, fmt.Sprintf("missing intent on: %s", ime.Key))
roachpb.RETRY_ASYNC_WRITE_FAILURE, redact.Sprintf("missing intent on: %s", ime.Key))
retryErr := roachpb.NewErrorWithTxn(err, pErr.GetTxn())
retryErr.Index = pErr.Index
return retryErr
Expand Down
11 changes: 6 additions & 5 deletions pkg/kv/kvclient/kvcoord/txn_interceptor_span_refresher.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,7 +12,6 @@ package kvcoord

import (
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/settings"
Expand All @@ -21,6 +20,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/metric"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

const (
Expand Down Expand Up @@ -474,15 +474,16 @@ func newRetryErrorOnFailedPreemptiveRefresh(
if txn.WriteTooOld {
reason = roachpb.RETRY_WRITE_TOO_OLD
}
msg := "failed preemptive refresh"
msg := redact.StringBuilder{}
msg.SafeString("failed preemptive refresh")
if refreshErr != nil {
if refreshErr, ok := refreshErr.GetDetail().(*roachpb.RefreshFailedError); ok {
msg = fmt.Sprintf("%s due to a conflict: %s on key %s", msg, refreshErr.FailureReason(), refreshErr.Key)
msg.Printf(" due to a conflict: %s on key %s", refreshErr.FailureReason(), refreshErr.Key)
} else {
msg = fmt.Sprintf("%s - unknown error: %s", msg, refreshErr)
msg.Printf(" - unknown error: %s", refreshErr)
}
}
retryErr := roachpb.NewTransactionRetryError(reason, msg)
retryErr := roachpb.NewTransactionRetryError(reason, msg.RedactableString())
return roachpb.NewErrorWithTxn(retryErr, txn)
}

Expand Down
6 changes: 3 additions & 3 deletions pkg/kv/kvserver/batcheval/cmd_end_transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,6 @@ package batcheval
import (
"bytes"
"context"
"fmt"
"math"
"sync/atomic"
"time"
Expand All @@ -35,6 +34,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

func init() {
Expand Down Expand Up @@ -473,7 +473,7 @@ func IsEndTxnExceedingDeadline(commitTS hlc.Timestamp, deadline hlc.Timestamp) b
// reason and possibly an extra message to be used for the error.
func IsEndTxnTriggeringRetryError(
txn *roachpb.Transaction, args *roachpb.EndTxnRequest,
) (retry bool, reason roachpb.TransactionRetryReason, extraMsg string) {
) (retry bool, reason roachpb.TransactionRetryReason, extraMsg redact.RedactableString) {
// If we saw any WriteTooOldErrors, we must restart to avoid lost
// update anomalies.
if txn.WriteTooOld {
Expand All @@ -492,7 +492,7 @@ func IsEndTxnTriggeringRetryError(
// A transaction must obey its deadline, if set.
if !retry && IsEndTxnExceedingDeadline(txn.WriteTimestamp, args.Deadline) {
exceededBy := txn.WriteTimestamp.GoTime().Sub(args.Deadline.GoTime())
extraMsg = fmt.Sprintf(
extraMsg = redact.Sprintf(
"txn timestamp pushed too much; deadline exceeded by %s (%s > %s)",
exceededBy, txn.WriteTimestamp, args.Deadline)
retry, reason = true, roachpb.RETRY_COMMIT_DEADLINE_EXCEEDED
Expand Down
4 changes: 2 additions & 2 deletions pkg/kv/kvserver/batcheval/transaction.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,13 +13,13 @@ package batcheval
import (
"bytes"
"context"
"fmt"

"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// ErrTransactionUnsupported is returned when a non-transactional command is
Expand Down Expand Up @@ -50,7 +50,7 @@ func VerifyTransaction(
reason = roachpb.TransactionStatusError_REASON_TXN_COMMITTED
}
return roachpb.NewTransactionStatusError(reason,
fmt.Sprintf("cannot perform %s with txn status %v", args.Method(), h.Txn.Status))
redact.Sprintf("cannot perform %s with txn status %v", args.Method(), h.Txn.Status))
}
return nil
}
Expand Down
5 changes: 4 additions & 1 deletion pkg/kv/mock_transactional_sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/redact"
)

// MockTransactionalSender allows a function to be used as a TxnSender.
Expand Down Expand Up @@ -178,7 +179,9 @@ func (m *MockTransactionalSender) UpdateStateOnRemoteRetryableErr(
func (m *MockTransactionalSender) DisablePipelining() error { return nil }

// PrepareRetryableError is part of the client.TxnSender interface.
func (m *MockTransactionalSender) PrepareRetryableError(ctx context.Context, msg string) error {
func (m *MockTransactionalSender) PrepareRetryableError(
ctx context.Context, msg redact.RedactableString,
) error {
return roachpb.NewTransactionRetryWithProtoRefreshError(msg, m.txn.ID, *m.txn.Clone())
}

Expand Down
3 changes: 2 additions & 1 deletion pkg/kv/sender.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/redact"
)

// TxnType specifies whether a transaction is the root (parent)
Expand Down Expand Up @@ -255,7 +256,7 @@ type TxnSender interface {
// PrepareRetryableError generates a
// TransactionRetryWithProtoRefreshError with a payload initialized
// from this txn.
PrepareRetryableError(ctx context.Context, msg string) error
PrepareRetryableError(ctx context.Context, msg redact.RedactableString) error

// TestingCloneTxn returns a clone of the transaction's current
// proto. This is for use by tests only. Use
Expand Down
5 changes: 4 additions & 1 deletion pkg/kv/txn.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/uuid"
"github.com/cockroachdb/errors"
"github.com/cockroachdb/redact"
)

// asyncRollbackTimeout is the context timeout during rollback() for a client
Expand Down Expand Up @@ -1419,7 +1420,9 @@ func (txn *Txn) SetFixedTimestamp(ctx context.Context, ts hlc.Timestamp) error {
// bumped to the extent that txn.ReadTimestamp is racheted up to txn.WriteTimestamp.
// TODO(andrei): This method should take in an up-to-date timestamp, but
// unfortunately its callers don't currently have that handy.
func (txn *Txn) GenerateForcedRetryableError(ctx context.Context, msg string) error {
func (txn *Txn) GenerateForcedRetryableError(
ctx context.Context, msg redact.RedactableString,
) error {
txn.mu.Lock()
defer txn.mu.Unlock()
now := txn.db.clock.NowAsClockTimestamp()
Expand Down
4 changes: 0 additions & 4 deletions pkg/roachpb/ambiguous_result_error.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,10 +78,6 @@ func (e *AmbiguousResultError) unwrapOrDefault() error {
return cause
}

func (e *AmbiguousResultError) message(_ *Error) string {
return fmt.Sprintf("result is ambiguous: %v", e.unwrapOrDefault())
}

// Type is part of the ErrorDetailInterface.
func (e *AmbiguousResultError) Type() ErrorDetailType {
return AmbiguousResultErrType
Expand Down
Loading

0 comments on commit 3e1cdb0

Please sign in to comment.