diff --git a/docs/generated/eventlog.md b/docs/generated/eventlog.md index 17cfa5fa35a0..af4d699af221 100644 --- a/docs/generated/eventlog.md +++ b/docs/generated/eventlog.md @@ -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 | @@ -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 | @@ -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 | @@ -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 | @@ -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 | @@ -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 | diff --git a/pkg/sql/exec_log.go b/pkg/sql/exec_log.go index 8acf25f4962b..f789c6c86e89 100644 --- a/pkg/sql/exec_log.go +++ b/pkg/sql/exec_log.go @@ -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 @@ -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() diff --git a/pkg/sql/telemetry_logging_test.go b/pkg/sql/telemetry_logging_test.go index f1b485ae15f9..df20044a295e 100644 --- a/pkg/sql/telemetry_logging_test.go +++ b/pkg/sql/telemetry_logging_test.go @@ -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. @@ -243,6 +244,23 @@ 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 { @@ -250,7 +268,10 @@ func TestTelemetryLogging(t *testing.T) { 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) + } } } @@ -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) + } + }) } } diff --git a/pkg/util/log/eventpb/json_encode_generated.go b/pkg/util/log/eventpb/json_encode_generated.go index 7090cb1e274d..681ebbada3d9 100644 --- a/pkg/util/log/eventpb/json_encode_generated.go +++ b/pkg/util/log/eventpb/json_encode_generated.go @@ -1710,9 +1710,7 @@ func (m *CommonSQLExecDetails) AppendJSONFields(printComma bool, b redact.Redact } printComma = true b = append(b, "\"ErrorText\":\""...) - b = append(b, redact.StartMarker()...) - b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(redact.EscapeMarkers([]byte(m.ErrorText))))) - b = append(b, redact.EndMarker()...) + b = redact.RedactableBytes(jsonbytes.EncodeString([]byte(b), string(m.ErrorText))) b = append(b, '"') } diff --git a/pkg/util/log/eventpb/sql_audit_events.proto b/pkg/util/log/eventpb/sql_audit_events.proto index 504b38a0e3ac..51b1844e8408 100644 --- a/pkg/util/log/eventpb/sql_audit_events.proto +++ b/pkg/util/log/eventpb/sql_audit_events.proto @@ -30,7 +30,7 @@ message CommonSQLExecDetails { // The SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. string sqlstate = 3 [(gogoproto.customname) = "SQLSTATE", (gogoproto.jsontag) = ",omitempty", (gogoproto.moretags) = "redact:\"nonsensitive\""]; // The text of the error if any. - string error_text = 4 [(gogoproto.jsontag) = ",omitempty"]; + string error_text = 4 [(gogoproto.jsontag) = ",omitempty", (gogoproto.customtype) = "github.com/cockroachdb/redact.RedactableString", (gogoproto.nullable) = false, (gogoproto.moretags) = "redact:\"mixed\""]; // Age of the query in milliseconds. float age = 5 [(gogoproto.jsontag) = ",omitempty"]; // Number of retries, when the txn was reretried automatically by the server.