Skip to content

Commit

Permalink
Merge #59110
Browse files Browse the repository at this point in the history
59110: sql: convert SQL audit and execution log to a structured format r=jordanlewis a=knz

First commit from #59108.
Fixes #59105. 
Fixes #58999. 

Example new format for the query log:
```
I210118 17:20:32.675052 2009 10@util/log/event_log.go:32  [n1,client=127.0.0.1:11362,hostssl,user=demo] Structured entry:
{"Timestamp":1610990432674339786,
 "EventType":"slow_query",
 "Statement":"SELECT * FROM \"\".\"\".t WHERE x = 10",
 "User":"demo",
 "ApplicationName":"$ cockroach demo",
 "ExecMode":"exec",
 "Age":0.790742,
 "FullTableScan":true}
 ```

Example new format, for audit events:
```
I210122 16:33:01.297735 2012 8@util/log/event_log.go:32 [n1,client=127.0.0.1:59820,hostssl,user=demo] Structured entry:
{"Timestamp":1611333181296915618,
 "EventType":"sensitive_table_access",
 "Statement":"INSERT INTO \"\".\"\".helloworld(abc) VALUES (1)",
 "User":"demo",
 "DescriptorID":53,
 "ApplicationName":"$ cockroach demo",
 "ExecMode":"exec",
 "NumRows":1,
 "Age":0.895012,
 "TableName":"t.public.helloworld",
 "AccessMode":"rw"}
```

(Audit events have more fields, specifically `DescriptorID`,
`TableName` and `AccessMode`, because they pertain to specific
objects for which audit logging has been triggered.)

Release note (sql change): CockroachDB now uses
a structured logging format for the SQL audit, execution and
query logs. See the reference documentation for details.

Of note, audit and execution logs now also include information
about whether a query plan contain full index scans. Previously,
this information was only included in theslow query log.

Release note (backward-incompatible change): The logging format for
SQL audit, execution and query logs has changed, from a crude
space-delimited format to JSON. To opt out of this new behavior and
restore the pre-v21.1 logging format, you can set the cluster setting
`sql.log.unstructured_entries.enabled` to true.

Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
craig[bot] and knz committed Jan 23, 2021
2 parents 7a5d3d0 + 253bff3 commit 2b2d350
Show file tree
Hide file tree
Showing 19 changed files with 2,587 additions and 132 deletions.
1 change: 1 addition & 0 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -1549,6 +1549,7 @@ EVENTLOG_PROTOS = \
pkg/util/log/eventpb/role_events.proto \
pkg/util/log/eventpb/zone_events.proto \
pkg/util/log/eventpb/session_events.proto \
pkg/util/log/eventpb/sql_audit_events.proto \
pkg/util/log/eventpb/cluster_events.proto

docs/generated/eventlog.md: pkg/util/log/eventpb/gen.go $(EVENTLOG_PROTOS) | bin/.go_protobuf_sources
Expand Down
224 changes: 224 additions & 0 deletions docs/generated/eventlog.md

Large diffs are not rendered by default.

13 changes: 8 additions & 5 deletions pkg/cli/interactive_tests/test_audit_log.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -23,22 +23,25 @@ start_test "Check that statements start being logged synchronously if auditing i
send "ALTER TABLE helloworld EXPERIMENTAL_AUDIT SET READ WRITE;\r"
eexpect root@
# check that the audit change itself is recorded.
system "grep -q 'helloworld.*:READWRITE.*ALTER TABLE.*OK' $logfile"
# Note: we really would like to check for redaction markers here, alas this grep
# command is running inside the acceptance image and does not know about UTF-8.
# So we use an imprecise match instead.
system "grep -q 'sensitive_table_access.*ALTER TABLE.*helloworld.*\"TableName\":\".*t.public.helloworld.*\",\"AccessMode\":\"rw\"' $logfile"
send "SELECT * FROM helloworld;\r"
eexpect root@
system "grep -q 'helloworld.*:READ}.*SELECT.*OK' $logfile"
system "grep -q 'sensitive_table_access.*SELECT.*helloworld.*\"TableName\":\".*t.public.helloworld.*\",\"AccessMode\":\"r\"' $logfile"
end_test

start_test "Check that write statements are logged differently"
send "INSERT INTO helloworld VALUES(456);\r"
eexpect root@
system "grep -q 'helloworld.*:READWRITE.*INSERT.*OK' $logfile"
system "grep -q 'sensitive_table_access.*INSERT.*helloworld.*AccessMode\":\"rw\"' $logfile"
end_test

start_test "Check that errors get logged too"
send "SELECT nonexistent FROM helloworld;\r"
eexpect root@
system "grep -q 'helloworld.*:READ}.*SELECT.*ERROR' $logfile"
system "grep -q 'sensitive_table_access.*SELECT.*nonexistent.*SQLSTATE.*42703.*\"AccessMode\":\"r\"' $logfile"
end_test

# Flush and truncate the logs. The test below must not see the log entries that
Expand All @@ -52,7 +55,7 @@ system "if grep -q helloworld $logfile; then false; fi"
start_test "Check that audit removal is logged too"
send "ALTER TABLE helloworld EXPERIMENTAL_AUDIT SET OFF;\r"
eexpect root@
system "grep 'helloworld.*:READWRITE.*ALTER TABLE.*SET OFF.*OK' $logfile"
system "grep -q 'sensitive_table_access.*ALTER TABLE.*helloworld.*SET OFF.*AccessMode\":\"rw\"' $logfile"
end_test

interrupt
Expand Down
4 changes: 3 additions & 1 deletion pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -825,7 +825,9 @@ func (n *Node) recordJoinEvent(ctx context.Context) {
int32(n.Descriptor.NodeID),
int32(n.Descriptor.NodeID),
true, /* skipExternalLog - we already call log.StructuredEvent above */
event)
event,
false, /* onlyLog */
)
}); err != nil {
log.Warningf(ctx, "%s: unable to log event %v: %v", n, event, err)
} else {
Expand Down
4 changes: 3 additions & 1 deletion pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -2137,7 +2137,9 @@ func (s *Server) Decommission(
txn,
int32(nodeID), int32(s.NodeID()),
true, /* skipExternalLog - we already call log.StructuredEvent above */
event)
event,
false, /* onlyLog */
)
}); err != nil {
log.Ops.Errorf(ctx, "unable to record event: %+v: %+v", event, err)
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -412,6 +412,7 @@ go_test(
"drop_helpers_test.go",
"drop_test.go",
"err_count_test.go",
"event_log_test.go",
"explain_bundle_test.go",
"explain_test.go",
"explain_tree_test.go",
Expand Down Expand Up @@ -555,6 +556,7 @@ go_test(
"//pkg/util/json",
"//pkg/util/leaktest",
"//pkg/util/log",
"//pkg/util/log/eventpb",
"//pkg/util/metric",
"//pkg/util/mon",
"//pkg/util/protoutil",
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -567,9 +567,12 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er
evalCtx.SessionData.User(),
evalCtx.SessionData.ApplicationName,
details.Statement,
nil, /* no placeholders known at this point */
&eventpb.CreateStatistics{
TableName: details.FQTableName,
})
},
true, /* writeToEventLog */
)
})
}

Expand Down
51 changes: 48 additions & 3 deletions pkg/sql/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,13 +30,34 @@ import (
// statement.
func (p *planner) logEvent(
ctx context.Context, descID descpb.ID, event eventpb.EventPayload,
) error {
return p.logEventWithSystemEventLogOption(ctx, descID, event, true /* writeToEventLog */)
}

func (p *planner) logEventOnlyExternally(
ctx context.Context, descID descpb.ID, event eventpb.EventPayload,
) {
// The API contract for logEventWithSystemEventLogOption() is that it returns
// no error when system.eventlog is not written to.
_ = p.logEventWithSystemEventLogOption(ctx, descID, event, false /* writeToEventLog */)
}

// logEventWithSystemEventLogOption is like logEvent() but it gives
// control to the caller as to whether the entry is written into
// system.eventlog.
//
// If writeToEventLog is false, this function guarantees that it
// returns no error.
func (p *planner) logEventWithSystemEventLogOption(
ctx context.Context, descID descpb.ID, event eventpb.EventPayload, writeToEventLog bool,
) error {
// Compute the common fields from data already known to the planner.
user := p.User()
stmt := tree.AsStringWithFQNames(p.stmt.AST, p.extendedEvalCtx.EvalContext.Annotations)
pl := p.extendedEvalCtx.EvalContext.Placeholders.Values
appName := p.SessionData().ApplicationName

return logEventInternalForSQLStatements(ctx, p.extendedEvalCtx.ExecCfg, p.txn, descID, user, appName, stmt, event)
return logEventInternalForSQLStatements(ctx, p.extendedEvalCtx.ExecCfg, p.txn, descID, user, appName, stmt, pl, event, writeToEventLog)
}

// logEventInternalForSchemaChange emits a cluster event in the
Expand Down Expand Up @@ -67,14 +88,19 @@ func logEventInternalForSchemaChanges(
int32(descID),
int32(execCfg.NodeID.SQLInstanceID()),
false, /* skipExternalLog */
event)
event,
false, /* onlyLog */
)
}

// logEventInternalForSQLStatements emits a cluster event on behalf of
// a SQL statement, when the point where the event is emitted does not
// have access to a (*planner) and the current statement metadata.
//
// Note: usage of this interface should be minimized.
//
// If writeToEventLog is false, this function guarantees that it
// returns no error.
func logEventInternalForSQLStatements(
ctx context.Context,
execCfg *ExecutorConfig,
Expand All @@ -83,7 +109,9 @@ func logEventInternalForSQLStatements(
user security.SQLUsername,
appName string,
stmt string,
placeholders tree.QueryArguments,
event eventpb.EventPayload,
writeToEventLog bool,
) error {
// Inject the common fields into the payload provided by the caller.
event.CommonDetails().Timestamp = txn.ReadTimestamp().WallTime
Expand All @@ -96,14 +124,22 @@ func logEventInternalForSQLStatements(
m.ApplicationName = appName
m.User = user.Normalized()
m.DescriptorID = uint32(descID)
if len(placeholders) > 0 {
m.PlaceholderValues = make([]string, len(placeholders))
for idx, val := range placeholders {
m.PlaceholderValues[idx] = val.String()
}
}

// Delegate the storing of the event to the regular event logic.
return InsertEventRecord(ctx, execCfg.InternalExecutor,
txn,
int32(descID),
int32(execCfg.NodeID.SQLInstanceID()),
false, /* skipExternalLog */
event)
event,
!writeToEventLog,
)
}

var eventLogEnabled = settings.RegisterBoolSetting(
Expand All @@ -124,13 +160,17 @@ var eventLogEnabled = settings.RegisterBoolSetting(
//
// Note: the targetID and reportingID columns are deprecated and
// should be removed after v21.1 is released.
//
// If onlyLog is set, this function guarantees that it returns no
// error.
func InsertEventRecord(
ctx context.Context,
ex *InternalExecutor,
txn *kv.Txn,
targetID, reportingID int32,
skipExternalLog bool,
info eventpb.EventPayload,
onlyLog bool,
) error {
eventType := eventpb.GetEventTypeName(info)

Expand All @@ -142,6 +182,11 @@ func InsertEventRecord(
return errors.AssertionFailedf("programming error: timestamp field in event not populated: %T", info)
}

if onlyLog {
log.StructuredEvent(ctx, info)
return nil
}

// Ensure that the external logging sees the event when the
// transaction commits.
txn.AddCommitTrigger(func(ctx context.Context) {
Expand Down
100 changes: 100 additions & 0 deletions pkg/sql/event_log_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2020 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 sql_test

import (
"context"
"encoding/json"
"math"
"reflect"
"regexp"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"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"
)

func TestStructuredEventLogging(t *testing.T) {
defer leaktest.AfterTest(t)()

// We really need to have the logs go to files, so that -show-logs
// does not break the "authlog" directives.
sc := log.ScopeWithoutShowLogs(t)
defer sc.Close(t)

ctx := context.Background()

s, conn, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)

testStartTs := timeutil.Now()

// Make a prepared statement that changes a cluster setting:
// - we want a prepared statement to verify that the reporting of
// placeholders works during EXECUTE.
// - we don't care about the particular cluster setting; any
// setting that does not otherwise impact the test's semantics
// will do.
const setStmt = `SET CLUSTER SETTING "sql.defaults.default_int_size" = $1`
if _, err := conn.ExecContext(ctx,
`PREPARE a(INT) AS `+setStmt,
); err != nil {
t.Fatal(err)
}
// Run the prepared statement. This triggers a structured entry
// for the cluster setting change.
if _, err := conn.ExecContext(ctx, `EXECUTE a(8)`); err != nil {
t.Fatal(err)
}

// Ensure that the entries hit the OS so they can be read back below.
log.Flush()

entries, err := log.FetchEntriesFromFiles(testStartTs.UnixNano(),
math.MaxInt64, 10000, execLogRe, log.WithMarkedSensitiveData)
if err != nil {
t.Fatal(err)
}

foundEntry := false
for _, e := range entries {
if !strings.Contains(e.Message, "set_cluster_setting") {
continue
}
foundEntry = true
// TODO(knz): Remove this when crdb-v2 becomes the new format.
e.Message = strings.TrimPrefix(e.Message, "Structured entry:")
// crdb-v2 starts json with an equal sign.
e.Message = strings.TrimPrefix(e.Message, "=")
jsonPayload := []byte(e.Message)
var ev eventpb.SetClusterSetting
if err := json.Unmarshal(jsonPayload, &ev); err != nil {
t.Errorf("unmarshalling %q: %v", e.Message, err)
}
if expected := string(redact.Sprint(setStmt)); ev.Statement != expected {
t.Errorf("wrong statement: expected %q, got %q", expected, ev.Statement)
}
if expected := []string{string(redact.Sprint("8"))}; !reflect.DeepEqual(expected, ev.PlaceholderValues) {
t.Errorf("wrong placeholders: expected %+v, got %+v", expected, ev.PlaceholderValues)
}
}
if !foundEntry {
t.Error("structured entry for set_cluster_setting not found in log")
}
}

var execLogRe = regexp.MustCompile(`event_log.go`)
Loading

0 comments on commit 2b2d350

Please sign in to comment.