Skip to content

Commit

Permalink
sql/event_log: fix bug with verbose logging
Browse files Browse the repository at this point in the history
This patch fixes a bug introduced when query logging
started using structured events: the ability to automatically
copy all the execution events to the DEV channel when
setting the `vmodule` setting to `exec_log=2` (or above).

In addition to fixing that bug, the following new vmodule-based
abilities are added:

- events for DDL statements and others that call `logEvent()` can now
  be collected in the DEV channel by using the name of the source file
  where they were generated as filter (e.g. `vmodule=create_table=2` for
  the CREATE TABLE events.
- events of other kinds can be collected in the DEV channel
  by setting `vmodule=event_log=2`.

(Note a subtle difference between `vmodule=create_table=2` and
`vmodule=exec_log=2`: the former emits the event to the DEV channel
while the stmt is executed; the latter emits the event after the stmt
completes. If both are enabled, TWO events are sent to the DEV channel.)

Since all the vmodule filtering options are subject to change without
notice between versions, we do not wish to document these nuances.
For this reason, the release note below is left blank.

Release note: None
  • Loading branch information
knz committed May 14, 2021
1 parent fc5041a commit c8b0ab2
Show file tree
Hide file tree
Showing 5 changed files with 70 additions and 14 deletions.
4 changes: 2 additions & 2 deletions pkg/server/node.go
Original file line number Diff line number Diff line change
Expand Up @@ -824,8 +824,8 @@ func (n *Node) recordJoinEvent(ctx context.Context) {
return sql.InsertEventRecord(ctx, n.sqlExec,
txn,
int32(n.Descriptor.NodeID), /* reporting ID: the node where the event is logged */
sql.LogToSystemTable, /* LogEventDestination: we already call log.StructuredEvent above */
int32(n.Descriptor.NodeID), /* target ID: the node that is joining (ourselves) */
sql.LogToSystemTable|sql.LogToDevChannelIfVerbose, /* LogEventDestination: we already call log.StructuredEvent above */
int32(n.Descriptor.NodeID), /* target ID: the node that is joining (ourselves) */
event,
)
}); err != nil {
Expand Down
6 changes: 3 additions & 3 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -2346,9 +2346,9 @@ func (s *Server) Decommission(
ctx,
s.sqlServer.execCfg.InternalExecutor,
txn,
int32(s.NodeID()), /* reporting ID: the node where the event is logged */
sql.LogToSystemTable, /* we already call log.StructuredEvent above */
int32(nodeID), /* target ID: the node that we wee a membership change for */
int32(s.NodeID()), /* reporting ID: the node where the event is logged */
sql.LogToSystemTable|sql.LogToDevChannelIfVerbose, /* we already call log.StructuredEvent above */
int32(nodeID), /* target ID: the node that we wee a membership change for */
event,
)
}); err != nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/create_stats.go
Original file line number Diff line number Diff line change
Expand Up @@ -577,6 +577,7 @@ func (r *createStatsResumer) Resume(ctx context.Context, execCtx interface{}) er
return evalCtx.ExecCfg.DB.Txn(ctx, func(ctx context.Context, txn *kv.Txn) error {
return logEventInternalForSQLStatements(ctx,
evalCtx.ExecCfg, txn,
0, /* depth: use event_log=2 for vmodule filtering */
eventLogOptions{dst: LogEverywhere},
sqlEventCommonExecPayload{
user: evalCtx.SessionData.User(),
Expand Down
55 changes: 53 additions & 2 deletions pkg/sql/event_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -135,6 +135,7 @@ func (p *planner) logEvent(
ctx context.Context, descID descpb.ID, event eventpb.EventPayload,
) error {
return p.logEventsWithOptions(ctx,
2, /* depth: use caller location */
eventLogOptions{dst: LogEverywhere},
eventLogEntry{targetID: int32(descID), event: event})
}
Expand All @@ -145,6 +146,7 @@ func (p *planner) logEvent(
// processed using only one write batch (and thus lower latency).
func (p *planner) logEvents(ctx context.Context, entries ...eventLogEntry) error {
return p.logEventsWithOptions(ctx,
2, /* depth: use caller location */
eventLogOptions{dst: LogEverywhere},
entries...)
}
Expand All @@ -153,6 +155,15 @@ func (p *planner) logEvents(ctx context.Context, entries ...eventLogEntry) error
type eventLogOptions struct {
// Where to emit the log event to.
dst LogEventDestination

// By default, a copy of each structured event is sent to the DEV
// channel (in addition to its default, nominal channel) if the
// vmodule filter is set to 2 or higher for the source file where
// the event call originates.
//
// If verboseTraceLevel is non-zero, its value is used as value for
// the vmodule filter. See exec_log for an example use.
verboseTraceLevel log.Level
}

// logEventsWithOptions is like logEvent() but it gives control to the
Expand All @@ -161,7 +172,7 @@ type eventLogOptions struct {
// If opts.dst does not include LogToSystemTable, this function is
// guaranteed to not return an error.
func (p *planner) logEventsWithOptions(
ctx context.Context, opts eventLogOptions, entries ...eventLogEntry,
ctx context.Context, depth int, opts eventLogOptions, entries ...eventLogEntry,
) error {
commonPayload := sqlEventCommonExecPayload{
user: p.User(),
Expand All @@ -172,6 +183,7 @@ func (p *planner) logEventsWithOptions(
}
return logEventInternalForSQLStatements(ctx,
p.extendedEvalCtx.ExecCfg, p.txn,
1+depth,
opts,
commonPayload,
entries...)
Expand Down Expand Up @@ -199,10 +211,15 @@ func logEventInternalForSchemaChanges(
m.MutationID = uint32(mutationID)

// Delegate the storing of the event to the regular event logic.
//
// We use depth=1 because the caller of this function typically
// wraps the call in a db.Txn() callback, which confuses the vmodule
// filtering. Easiest is to pretend the event is sourced here.
return insertEventRecords(
ctx, execCfg.InternalExecutor,
txn,
int32(execCfg.NodeID.SQLInstanceID()), /* reporter ID */
1, /* depth: use this function as origin */
eventLogOptions{dst: LogEverywhere},
eventLogEntry{
targetID: int32(descID),
Expand Down Expand Up @@ -233,6 +250,7 @@ func logEventInternalForSQLStatements(
ctx context.Context,
execCfg *ExecutorConfig,
txn *kv.Txn,
depth int,
opts eventLogOptions,
commonPayload sqlEventCommonExecPayload,
entries ...eventLogEntry,
Expand Down Expand Up @@ -269,6 +287,7 @@ func logEventInternalForSQLStatements(
return insertEventRecords(ctx,
execCfg.InternalExecutor, txn,
int32(execCfg.NodeID.SQLInstanceID()), /* reporter ID */
1+depth, /* depth */
opts, /* eventLogOptions */
entries..., /* ...eventLogEntry */
)
Expand Down Expand Up @@ -301,10 +320,15 @@ func LogEventForJobs(
m.Description = payload.Description

// Delegate the storing of the event to the regular event logic.
//
// We use depth=1 because the caller of this function typically
// wraps the call in a db.Txn() callback, which confuses the vmodule
// filtering. Easiest is to pretend the event is sourced here.
return insertEventRecords(
ctx, execCfg.InternalExecutor,
txn,
int32(execCfg.NodeID.SQLInstanceID()), /* reporter ID */
1, /* depth: use this function for vmodule filtering */
eventLogOptions{dst: LogEverywhere},
eventLogEntry{event: event},
)
Expand Down Expand Up @@ -332,9 +356,13 @@ const (
// LogExternally makes InsertEventRecord write the event(s) to the
// external logs.
LogExternally
// LogToDevChannelIfVerbose makes InsertEventRecord copy
// the structured event to the DEV logging channel
// if the vmodule filter for the log call is set high enough.
LogToDevChannelIfVerbose

// LogEverywhere logs to all the possible outputs.
LogEverywhere LogEventDestination = LogExternally | LogToSystemTable
LogEverywhere LogEventDestination = LogExternally | LogToSystemTable | LogToDevChannelIfVerbose
)

// InsertEventRecord inserts a single event into the event log as part
Expand All @@ -350,7 +378,11 @@ func InsertEventRecord(
targetID int32,
info eventpb.EventPayload,
) error {
// We use depth=1 because the caller of this function typically
// wraps the call in a db.Txn() callback, which confuses the vmodule
// filtering. Easiest is to pretend the event is sourced here.
return insertEventRecords(ctx, ex, txn, reportingID,
1, /* depth: use this function */
eventLogOptions{dst: dst},
eventLogEntry{targetID: targetID, event: info})
}
Expand All @@ -370,6 +402,7 @@ func insertEventRecords(
ex *InternalExecutor,
txn *kv.Txn,
reportingID int32,
depth int,
opts eventLogOptions,
entries ...eventLogEntry,
) error {
Expand All @@ -386,6 +419,24 @@ func insertEventRecords(
}
}

if opts.dst.hasFlag(LogToDevChannelIfVerbose) {
// Emit a copy of the structured to the DEV channel when the
// vmodule setting matches.
level := log.Level(2)
if opts.verboseTraceLevel != 0 {
// Caller has overridden the level at which which log to the
// trace.
level = opts.verboseTraceLevel
}
if log.VDepth(level, depth) {
// The VDepth() call ensures that we are matching the vmodule
// setting to where the depth is equal to 1 in the caller stack.
for i := range entries {
log.InfofDepth(ctx, depth, "SQL event: target %d, payload %+v", entries[i].targetID, entries[i].event)
}
}
}

// If we only want to log externally and not write to the events table, early exit.
loggingToSystemTable := opts.dst.hasFlag(LogToSystemTable) && eventLogSystemTableEnabled.Get(&ex.s.cfg.Settings.SV)
if !loggingToSystemTable {
Expand Down
18 changes: 11 additions & 7 deletions pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -276,11 +276,6 @@ func (p *planner) maybeLogStatementInternal(
TxnCounter: uint32(txnCounter),
}

if logV {
// Copy to the debug log / trace.
log.VEventf(ctx, execType.vLevel(), "%+v", execDetails)
}

if auditEventsDetected {
// TODO(knz): re-add the placeholders and age into the logging event.
entries := make([]eventLogEntry, len(p.curPlan.auditEvents))
Expand Down Expand Up @@ -334,8 +329,16 @@ func (p *planner) maybeLogStatementInternal(
}
}

if logExecuteEnabled {
p.logEventsOnlyExternally(ctx,
if logExecuteEnabled || logV {
p.logEventsWithOptions(ctx,
1, /* depth */
eventLogOptions{
// We pass LogToDevChannelIfVerbose because we have a log.V
// request for this file, which means the operator wants to
// see a copy of the execution on the DEV Channel.
dst: LogExternally | LogToDevChannelIfVerbose,
verboseTraceLevel: execType.vLevel(),
},
eventLogEntry{event: &eventpb.QueryExecute{CommonSQLExecDetails: execDetails}})
}

Expand All @@ -348,6 +351,7 @@ func (p *planner) logEventsOnlyExternally(ctx context.Context, entries ...eventL
// The API contract for logEventsWithOptions() is that it returns
// no error when system.eventlog is not written to.
_ = p.logEventsWithOptions(ctx,
2, /* depth: we want to use the caller location */
eventLogOptions{dst: LogExternally},
entries...)
}
Expand Down

0 comments on commit c8b0ab2

Please sign in to comment.