Skip to content

Commit

Permalink
Merge #83807
Browse files Browse the repository at this point in the history
83807: sql, telemetry: show correctly redacted error message r=andreimatei,rafiss,abarganier a=dhartunian

Previously, the error string from an executed SQL statement was treated
as a `string` type which would get fully redacted on its way out to the
telemetry log.

This change writes the `error` type into a `RedactableString` which
preserves the redaction in the error as intended. This will preserve the
template string which we consider safe by default, for instance.

The `CommonSQLExecDetails` type in `sql_audit_events.proto` has been
modified to have the `error_text` field represented as a
`RedactableString` which helps us maintain the preservation of redaction
markers.

Resolves: #78353

Release note (ops change): Telemetry logs will now display more finely
redacted error messages from sql execution. Previously, the entire error
string was fully redacted.

Co-authored-by: David Hartunian <[email protected]>
  • Loading branch information
craig[bot] and dhartunian committed Aug 1, 2022
2 parents b83e83b + 53f6a05 commit 94b9091
Show file tree
Hide file tree
Showing 5 changed files with 157 additions and 129 deletions.
12 changes: 6 additions & 6 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -389,7 +389,7 @@ is directly or indirectly a member of the admin role) executes a query.
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down Expand Up @@ -423,7 +423,7 @@ a table marked as audited.
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down Expand Up @@ -464,7 +464,7 @@ and the cluster setting `sql.trace.log_statement_execute` is set.
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down Expand Up @@ -2064,7 +2064,7 @@ set to a non-zero value, AND
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down Expand Up @@ -2183,7 +2183,7 @@ the "slow query" condition.
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down Expand Up @@ -2489,7 +2489,7 @@ contains common SQL event/execution details.
| `ExecMode` | How the statement was being executed (exec/prepare, etc.) | no |
| `NumRows` | Number of rows returned. For mutation statements (INSERT, etc) that do not produce result rows, this field reports the number of rows affected. | no |
| `SQLSTATE` | The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | The text of the error if any. | yes |
| `ErrorText` | The text of the error if any. | partially |
| `Age` | Age of the query in milliseconds. | no |
| `NumRetries` | Number of retries, when the txn was reretried automatically by the server. | no |
| `FullTableScan` | Whether the query contains a full table scan. | no |
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/log/eventpb"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/redact"
)

// This file contains facilities to report SQL activities to separate
Expand Down Expand Up @@ -213,9 +214,9 @@ func (p *planner) maybeLogStatementInternal(
age := float32(queryDuration.Nanoseconds()) / 1e6
// The text of the error encountered, if the query did in fact end
// in error.
execErrStr := ""
var execErrStr redact.RedactableString
if err != nil {
execErrStr = err.Error()
execErrStr = redact.Sprint(err)
}
// The type of execution context (execute/prepare).
lbl := execType.logLabel()
Expand Down
263 changes: 146 additions & 117 deletions pkg/sql/telemetry_logging_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -137,6 +137,7 @@ func TestTelemetryLogging(t *testing.T) {
expectedStatsAvailable bool
expectedRead bool
expectedWrite bool
expectedErr string // Empty string means no error is expected.
}{
{
// Test case with statement that is not of type DML.
Expand Down Expand Up @@ -243,14 +244,34 @@ func TestTelemetryLogging(t *testing.T) {
expectedRead: true,
expectedWrite: true,
},
// Not of type DML so not sampled
{
name: "sql-error",
query: "CREATE USER root;",
queryNoConstants: "CREATE USER root",
execTimestampsSeconds: []float64{9},
expectedLogStatement: `CREATE USER root`,
stubMaxEventFrequency: 1,
expectedSkipped: []int{0},
expectedUnredactedTags: []string{"client"},
expectedApplicationName: "telemetry-logging-test",
expectedFullScan: false,
expectedStatsAvailable: false,
expectedRead: false,
expectedWrite: false,
expectedErr: "a role/user named ‹root› already exists",
},
}

for _, tc := range testData {
telemetryMaxEventFrequency.Override(context.Background(), &s.ClusterSettings().SV, tc.stubMaxEventFrequency)
for _, execTimestamp := range tc.execTimestampsSeconds {
stubTime := timeutil.FromUnixMicros(int64(execTimestamp * 1e6))
st.setTime(stubTime)
db.Exec(t, tc.query)
_, err := db.DB.ExecContext(context.Background(), tc.query)
if err != nil && tc.expectedErr == "" {
t.Errorf("unexpected error executing query `%s`: %v", tc.query, err)
}
}
}

Expand Down Expand Up @@ -279,142 +300,150 @@ func TestTelemetryLogging(t *testing.T) {
}

for _, tc := range testData {
logCount := 0
expectedLogCount := len(tc.expectedSkipped)
// NB: FetchEntriesFromFiles delivers entries in reverse order.
for i := len(entries) - 1; i >= 0; i-- {
e := entries[i]
if strings.Contains(e.Message, tc.expectedLogStatement) {
if logCount == expectedLogCount {
t.Errorf("%s: found more than %d expected log entries", tc.name, expectedLogCount)
break
}
expectedSkipped := tc.expectedSkipped[logCount]
logCount++
if expectedSkipped == 0 {
if strings.Contains(e.Message, "SkippedQueries") {
t.Errorf("%s: expected no skipped queries, found:\n%s", tc.name, e.Message)
}
} else {
if expected := fmt.Sprintf(`"SkippedQueries":%d`, expectedSkipped); !strings.Contains(e.Message, expected) {
t.Errorf("%s: expected %s found:\n%s", tc.name, expected, e.Message)
t.Run(tc.name, func(t *testing.T) {
logCount := 0
expectedLogCount := len(tc.expectedSkipped)
// NB: FetchEntriesFromFiles delivers entries in reverse order.
for i := len(entries) - 1; i >= 0; i-- {
e := entries[i]
if strings.Contains(e.Message, tc.expectedLogStatement) {
if logCount == expectedLogCount {
t.Errorf("%s: found more than %d expected log entries", tc.name, expectedLogCount)
break
}
}
costRe := regexp.MustCompile("\"CostEstimate\":[0-9]*\\.[0-9]*")
if !costRe.MatchString(e.Message) {
t.Errorf("expected to find CostEstimate but none was found")
}
distRe := regexp.MustCompile("\"Distribution\":(\"full\"|\"local\")")
if !distRe.MatchString(e.Message) {
t.Errorf("expected to find Distribution but none was found")
}
// Match plan gist on any non-empty string value.
planGist := regexp.MustCompile("\"PlanGist\":(\"\\S+\")")
if !planGist.MatchString(e.Message) {
t.Errorf("expected to find PlanGist but none was found in: %s", e.Message)
}
// Match StatementID on any non-empty string value.
stmtID := regexp.MustCompile("\"StatementID\":(\"\\S+\")")
if !stmtID.MatchString(e.Message) {
t.Errorf("expected to find StatementID but none was found in: %s", e.Message)
}
// Match TransactionID on any non-empty string value.
txnID := regexp.MustCompile("\"TransactionID\":(\"\\S+\")")
if !txnID.MatchString(e.Message) {
t.Errorf("expected to find TransactionID but none was found in: %s", e.Message)
}
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)
expectedSkipped := tc.expectedSkipped[logCount]
logCount++
if expectedSkipped == 0 {
if strings.Contains(e.Message, "SkippedQueries") {
t.Errorf("%s: expected no skipped queries, found:\n%s", tc.name, e.Message)
}
} else {
if expected := fmt.Sprintf(`"SkippedQueries":%d`, expectedSkipped); !strings.Contains(e.Message, expected) {
t.Errorf("%s: expected %s found:\n%s", tc.name, expected, e.Message)
}
}
}
if !strings.Contains(e.Message, "\"ApplicationName\":\""+tc.expectedApplicationName+"\"") {
t.Errorf("expected to find unredacted Application Name: %s", tc.expectedApplicationName)
}
if !strings.Contains(e.Message, "\"SessionID\":\""+sessionID+"\"") {
t.Errorf("expected to find sessionID: %s", sessionID)
}
if !strings.Contains(e.Message, "\"Database\":\""+databaseName+"\"") {
t.Errorf("expected to find Database: %s", databaseName)
}
stmtFingerprintID := roachpb.ConstructStatementFingerprintID(tc.queryNoConstants, false, true, databaseName)
if !strings.Contains(e.Message, "\"StatementFingerprintID\":"+strconv.FormatUint(uint64(stmtFingerprintID), 10)) {
t.Errorf("expected to find StatementFingerprintID: %v", stmtFingerprintID)
}
maxFullScanRowsRe := regexp.MustCompile("\"MaxFullScanRowsEstimate\":[0-9]*")
foundFullScan := maxFullScanRowsRe.MatchString(e.Message)
if tc.expectedFullScan && !foundFullScan {
t.Errorf("expected to find MaxFullScanRowsEstimate but none was found in: %s", e.Message)
} else if !tc.expectedFullScan && foundFullScan {
t.Errorf("expected not to find MaxFullScanRowsEstimate but it was found in: %s", e.Message)
}
totalScanRowsRe := regexp.MustCompile("\"TotalScanRowsEstimate\":[0-9]*")
outputRowsRe := regexp.MustCompile("\"OutputRowsEstimate\":[0-9]*")
statsAvailableRe := regexp.MustCompile("\"StatsAvailable\":(true|false)")
nanosSinceStatsCollectedRe := regexp.MustCompile("\"NanosSinceStatsCollected\":[0-9]*")
if tc.expectedStatsAvailable {
if !totalScanRowsRe.MatchString(e.Message) {
t.Errorf("expected to find TotalScanRowsEstimate but none was found in: %s", e.Message)
costRe := regexp.MustCompile("\"CostEstimate\":[0-9]*\\.[0-9]*")
if !costRe.MatchString(e.Message) {
t.Errorf("expected to find CostEstimate but none was found")
}
if !outputRowsRe.MatchString(e.Message) {
t.Errorf("expected to find OutputRowsEstimate but none was found in: %s", e.Message)
distRe := regexp.MustCompile("\"Distribution\":(\"full\"|\"local\")")
if !distRe.MatchString(e.Message) {
t.Errorf("expected to find Distribution but none was found")
}
if !statsAvailableRe.MatchString(e.Message) {
t.Errorf("expected to find StatsAvailable but none was found in: %s", e.Message)
// Match plan gist on any non-empty string value.
planGist := regexp.MustCompile("\"PlanGist\":(\"\\S+\")")
if !planGist.MatchString(e.Message) {
t.Errorf("expected to find PlanGist but none was found in: %s", e.Message)
}
if !nanosSinceStatsCollectedRe.MatchString(e.Message) {
t.Errorf("expected to find NanosSinceStatsCollected but none was found in: %s", e.Message)
// Match StatementID on any non-empty string value.
stmtID := regexp.MustCompile("\"StatementID\":(\"\\S+\")")
if !stmtID.MatchString(e.Message) {
t.Errorf("expected to find StatementID but none was found in: %s", e.Message)
}
} else {
if totalScanRowsRe.MatchString(e.Message) {
t.Errorf("expected not to find TotalScanRowsEstimate but it was found in: %s", e.Message)
// Match TransactionID on any non-empty string value.
txnID := regexp.MustCompile("\"TransactionID\":(\"\\S+\")")
if !txnID.MatchString(e.Message) {
t.Errorf("expected to find TransactionID but none was found in: %s", e.Message)
}
if outputRowsRe.MatchString(e.Message) {
t.Errorf("expected not to find OutputRowsEstimate but it was found in: %s", e.Message)
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 statsAvailableRe.MatchString(e.Message) {
t.Errorf("expected not to find StatsAvailable but it was found in: %s", e.Message)
if !strings.Contains(e.Message, "\"ApplicationName\":\""+tc.expectedApplicationName+"\"") {
t.Errorf("expected to find unredacted Application Name: %s", tc.expectedApplicationName)
}
if nanosSinceStatsCollectedRe.MatchString(e.Message) {
t.Errorf("expected not to find NanosSinceStatsCollected but it was found in: %s", e.Message)
if !strings.Contains(e.Message, "\"SessionID\":\""+sessionID+"\"") {
t.Errorf("expected to find sessionID: %s", sessionID)
}
}
BytesReadRe := regexp.MustCompile("\"BytesRead\":[0-9]*")
RowsReadRe := regexp.MustCompile("\"RowsRead\":[0-9]*")
if tc.expectedRead {
if !BytesReadRe.MatchString(e.Message) {
t.Errorf("expected to find BytesRead but none was found in: %s", e.Message)
if !strings.Contains(e.Message, "\"Database\":\""+databaseName+"\"") {
t.Errorf("expected to find Database: %s", databaseName)
}
if !RowsReadRe.MatchString(e.Message) {
t.Errorf("expected to find RowsRead but none was found in: %s", e.Message)
stmtFingerprintID := roachpb.ConstructStatementFingerprintID(tc.queryNoConstants, tc.expectedErr != "", true, databaseName)
if !strings.Contains(e.Message, "\"StatementFingerprintID\":"+strconv.FormatUint(uint64(stmtFingerprintID), 10)) {
t.Errorf("expected to find StatementFingerprintID: %v", stmtFingerprintID)
}
} else {
if BytesReadRe.MatchString(e.Message) {
t.Errorf("expected not to find BytesRead but it was found in: %s", e.Message)
maxFullScanRowsRe := regexp.MustCompile("\"MaxFullScanRowsEstimate\":[0-9]*")
foundFullScan := maxFullScanRowsRe.MatchString(e.Message)
if tc.expectedFullScan && !foundFullScan {
t.Errorf("expected to find MaxFullScanRowsEstimate but none was found in: %s", e.Message)
} else if !tc.expectedFullScan && foundFullScan {
t.Errorf("expected not to find MaxFullScanRowsEstimate but it was found in: %s", e.Message)
}
if RowsReadRe.MatchString(e.Message) {
t.Errorf("expected not to find RowsRead but it was found in: %s", e.Message)
totalScanRowsRe := regexp.MustCompile("\"TotalScanRowsEstimate\":[0-9]*")
outputRowsRe := regexp.MustCompile("\"OutputRowsEstimate\":[0-9]*")
statsAvailableRe := regexp.MustCompile("\"StatsAvailable\":(true|false)")
nanosSinceStatsCollectedRe := regexp.MustCompile("\"NanosSinceStatsCollected\":[0-9]*")
if tc.expectedStatsAvailable {
if !totalScanRowsRe.MatchString(e.Message) {
t.Errorf("expected to find TotalScanRowsEstimate but none was found in: %s", e.Message)
}
if !outputRowsRe.MatchString(e.Message) {
t.Errorf("expected to find OutputRowsEstimate but none was found in: %s", e.Message)
}
if !statsAvailableRe.MatchString(e.Message) {
t.Errorf("expected to find StatsAvailable but none was found in: %s", e.Message)
}
if !nanosSinceStatsCollectedRe.MatchString(e.Message) {
t.Errorf("expected to find NanosSinceStatsCollected but none was found in: %s", e.Message)
}
} else {
if totalScanRowsRe.MatchString(e.Message) {
t.Errorf("expected not to find TotalScanRowsEstimate but it was found in: %s", e.Message)
}
if outputRowsRe.MatchString(e.Message) {
t.Errorf("expected not to find OutputRowsEstimate but it was found in: %s", e.Message)
}
if statsAvailableRe.MatchString(e.Message) {
t.Errorf("expected not to find StatsAvailable but it was found in: %s", e.Message)
}
if nanosSinceStatsCollectedRe.MatchString(e.Message) {
t.Errorf("expected not to find NanosSinceStatsCollected but it was found in: %s", e.Message)
}
}
BytesReadRe := regexp.MustCompile("\"BytesRead\":[0-9]*")
RowsReadRe := regexp.MustCompile("\"RowsRead\":[0-9]*")
if tc.expectedRead {
if !BytesReadRe.MatchString(e.Message) {
t.Errorf("expected to find BytesRead but none was found in: %s", e.Message)
}
if !RowsReadRe.MatchString(e.Message) {
t.Errorf("expected to find RowsRead but none was found in: %s", e.Message)
}
} else {
if BytesReadRe.MatchString(e.Message) {
t.Errorf("expected not to find BytesRead but it was found in: %s", e.Message)
}
if RowsReadRe.MatchString(e.Message) {
t.Errorf("expected not to find RowsRead but it was found in: %s", e.Message)
}

}
RowsWrittenRe := regexp.MustCompile("\"RowsWritten\":[0-9]*")
if tc.expectedWrite {
if !RowsWrittenRe.MatchString(e.Message) {
t.Errorf("expected to find RowsWritten but none was found in: %s", e.Message)
}
} else {
if RowsWrittenRe.MatchString(e.Message) {
t.Errorf("expected not to find RowsWritten but it was found in: %s", e.Message)
RowsWrittenRe := regexp.MustCompile("\"RowsWritten\":[0-9]*")
if tc.expectedWrite {
if !RowsWrittenRe.MatchString(e.Message) {
t.Errorf("expected to find RowsWritten but none was found in: %s", e.Message)
}
} else {
if RowsWrittenRe.MatchString(e.Message) {
t.Errorf("expected not to find RowsWritten but it was found in: %s", e.Message)
}
}
if tc.expectedErr != "" {
if !strings.Contains(e.Message, tc.expectedErr) {
t.Errorf("%s: missing error %s in message %s", tc.name, tc.expectedErr, e.Message)
break
}
}
}
}
}
if logCount != expectedLogCount {
t.Errorf("%s: expected %d log entries, found %d", tc.name, expectedLogCount, logCount)
}
if logCount != expectedLogCount {
t.Errorf("%s: expected %d log entries, found %d", tc.name, expectedLogCount, logCount)
}
})
}
}

Expand Down
Loading

0 comments on commit 94b9091

Please sign in to comment.