Skip to content

Commit

Permalink
sql: add a new optional secondary logger for internal slow queries
Browse files Browse the repository at this point in the history
Previously, all slow queries were logged to the slow query log,
regardless of whether they were internal or from the user. This made
it easy for user queries to get lost in the log.

This patch removes internal queries from the slow query log and adds a
new optional opt-in cluster setting,
`sql.log.slow_query.internal_queries.enabled`. When set to
true, all internal queries that have service latency above the slow
query log threshold will be logged to
`cockroach-sql-slow-internal-only.log`. This new setting is a no-op if
the slow query log hasn't been turned on, i.e there is
no latency threshold set.

Closes #50201

Release note (sql change): adds a new cluster setting
`sql.log.slow_query.internal` which when turned on in conjunction with
the slow query log causes internal queries to be logged to an internal
query only slow query log, at `cockroach-sql-slow-internal-only.log`.
Internal queries are no longer logged to the slow query log. This new
setting is opt-in, so default behavior is to not log internal slow
queries.
  • Loading branch information
arulajmani committed Aug 17, 2020
1 parent ea094e9 commit 71d1502
Show file tree
Hide file tree
Showing 4 changed files with 49 additions and 22 deletions.
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Expand Up @@ -48,6 +48,7 @@
<tr><td><code>sql.defaults.results_buffer.size</code></td><td>byte size</td><td><code>16 KiB</code></td><td>default size of the buffer that accumulates results for a statement or a batch of statements before they are sent to the client. This can be overridden on an individual connection with the 'results_buffer_size' parameter. Note that auto-retries generally only happen while no results have been delivered to the client, so reducing this size can increase the number of retriable errors a client receives. On the other hand, increasing the buffer size can increase the delay until the client receives the first result row. Updating the setting only affects new connections. Setting to 0 disables any buffering.</td></tr>
<tr><td><code>sql.defaults.serial_normalization</code></td><td>enumeration</td><td><code>rowid</code></td><td>default handling of SERIAL in table definitions [rowid = 0, virtual_sequence = 1, sql_sequence = 2]</td></tr>
<tr><td><code>sql.distsql.max_running_flows</code></td><td>integer</td><td><code>500</code></td><td>maximum number of concurrent flows that can be run on a node</td></tr>
<tr><td><code>sql.log.slow_query.internal_queries.enabled</code></td><td>boolean</td><td><code>false</code></td><td>when set to true, internal queries which exceed the slow query log threshold are logged to a separate log. Must have the slow query log enabled for this setting to have any effect.</td></tr>
<tr><td><code>sql.log.slow_query.latency_threshold</code></td><td>duration</td><td><code>0s</code></td><td>when set to non-zero, log statements whose service latency exceeds the threshold to a secondary logger on each node</td></tr>
<tr><td><code>sql.metrics.statement_details.dump_to_logs</code></td><td>boolean</td><td><code>false</code></td><td>dump collected statement statistics to node logs when periodically cleared</td></tr>
<tr><td><code>sql.metrics.statement_details.enabled</code></td><td>boolean</td><td><code>true</code></td><td>collect per-statement query statistics</td></tr>
Expand Down
12 changes: 8 additions & 4 deletions pkg/server/server_sql.go
Expand Up @@ -464,7 +464,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {

ExecLogger: log.NewSecondaryLogger(
loggerCtx, nil /* dirName */, "sql-exec",
true /* enableGc */, false /*forceSyncWrites*/, true, /* enableMsgCount */
true /* enableGc */, false /* forceSyncWrites */, true, /* enableMsgCount */
),

// Note: the auth logger uses sync writes because we don't want an
Expand All @@ -478,20 +478,23 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
// would be a good reason to invest into a syslog sink for logs.
AuthLogger: log.NewSecondaryLogger(
loggerCtx, nil /* dirName */, "auth",
true /* enableGc */, true /*forceSyncWrites*/, true, /* enableMsgCount */
true /* enableGc */, true /* forceSyncWrites */, true, /* enableMsgCount */
),

// AuditLogger syncs to disk for the same reason as AuthLogger.
AuditLogger: log.NewSecondaryLogger(
loggerCtx, cfg.AuditLogDirName, "sql-audit",
true /*enableGc*/, true /*forceSyncWrites*/, true, /* enableMsgCount */
true /* enableGc */, true /* forceSyncWrites */, true, /* enableMsgCount */
),

SlowQueryLogger: log.NewSecondaryLogger(
loggerCtx, nil, "sql-slow",
true /*enableGc*/, false /*forceSyncWrites*/, true, /* enableMsgCount */
true /* enableGc */, false /* forceSyncWrites */, true, /* enableMsgCount */
),

SlowInternalQueryLogger: log.NewSecondaryLogger(loggerCtx, nil, "sql-slow-internal-only",
true /* enableGc */, false /* forceSyncWrites */, true /* enableMsgCount */),

QueryCache: querycache.New(cfg.QueryCacheSize),
ProtectedTimestampProvider: cfg.protectedtsProvider,
ExternalIODirConfig: cfg.ExternalIODirConfig,
Expand All @@ -500,6 +503,7 @@ func newSQLServer(ctx context.Context, cfg sqlServerArgs) (*sqlServer, error) {
cfg.stopper.AddCloser(execCfg.ExecLogger)
cfg.stopper.AddCloser(execCfg.AuditLogger)
cfg.stopper.AddCloser(execCfg.SlowQueryLogger)
cfg.stopper.AddCloser(execCfg.SlowInternalQueryLogger)
cfg.stopper.AddCloser(execCfg.AuthLogger)

if sqlSchemaChangerTestingKnobs := cfg.TestingKnobs.SQLSchemaChanger; sqlSchemaChangerTestingKnobs != nil {
Expand Down
27 changes: 24 additions & 3 deletions pkg/sql/exec_log.go
Expand Up @@ -96,6 +96,14 @@ var slowQueryLogThreshold = settings.RegisterPublicNonNegativeDurationSettingWit
0,
)

var slowInternalQueryLogEnabled = settings.RegisterPublicBoolSetting(
"sql.log.slow_query.internal_queries.enabled",
"when set to true, internal queries which exceed the slow query log threshold "+
"are logged to a separate log. Must have the slow query log enabled for this "+
"setting to have any effect.",
false,
)

type executorType int

const (
Expand Down Expand Up @@ -136,6 +144,7 @@ func (p *planner) maybeLogStatementInternal(
logExecuteEnabled := logStatementsExecuteEnabled.Get(&p.execCfg.Settings.SV)
slowLogThreshold := slowQueryLogThreshold.Get(&p.execCfg.Settings.SV)
slowQueryLogEnabled := slowLogThreshold != 0
slowInternalQueryLogEnabled := slowInternalQueryLogEnabled.Get(&p.execCfg.Settings.SV)
auditEventsDetected := len(p.curPlan.auditEvents) != 0

if !logV && !logExecuteEnabled && !auditEventsDetected && !slowQueryLogEnabled {
Expand Down Expand Up @@ -190,9 +199,21 @@ func (p *planner) maybeLogStatementInternal(
lbl, appName, logTrigger, stmtStr, plStr, age, rows, auditErrStr, numRetries)
}
if slowQueryLogEnabled && queryDuration > slowLogThreshold {
logger := p.execCfg.SlowQueryLogger
logger.Logf(ctx, "%.3fms %s %q %s %q %s %d %q %d",
age, lbl, appName, logTrigger, stmtStr, plStr, rows, execErrStr, numRetries)
var logger *log.SecondaryLogger
// Non-internal queries are always logged to the slow query log.
if execType == executorTypeExec {
logger = p.execCfg.SlowQueryLogger
}
// Internal queries that surpass the slow query log threshold should only
// be logged to the slow-internal-only log if the cluster setting dictates.
if execType == executorTypeInternal && slowInternalQueryLogEnabled {
logger = p.execCfg.SlowInternalQueryLogger
}

if logger != nil {
logger.Logf(ctx, "%.3fms %s %q %s %q %s %d %q %d",
age, lbl, appName, logTrigger, stmtStr, plStr, rows, execErrStr, numRetries)
}
}
if logExecuteEnabled {
logger := p.execCfg.ExecLogger
Expand Down
31 changes: 16 additions & 15 deletions pkg/sql/exec_util.go
Expand Up @@ -661,21 +661,22 @@ type ExecutorConfig struct {
Clock *hlc.Clock
DistSQLSrv *distsql.ServerImpl
// StatusServer gives access to the Status service.
StatusServer serverpb.OptionalStatusServer
MetricsRecorder nodeStatusGenerator
SessionRegistry *SessionRegistry
SQLLivenessReader sqlliveness.Reader
JobRegistry *jobs.Registry
VirtualSchemas *VirtualSchemaHolder
DistSQLPlanner *DistSQLPlanner
TableStatsCache *stats.TableStatisticsCache
StatsRefresher *stats.Refresher
ExecLogger *log.SecondaryLogger
AuditLogger *log.SecondaryLogger
SlowQueryLogger *log.SecondaryLogger
AuthLogger *log.SecondaryLogger
InternalExecutor *InternalExecutor
QueryCache *querycache.C
StatusServer serverpb.OptionalStatusServer
MetricsRecorder nodeStatusGenerator
SessionRegistry *SessionRegistry
SQLLivenessReader sqlliveness.Reader
JobRegistry *jobs.Registry
VirtualSchemas *VirtualSchemaHolder
DistSQLPlanner *DistSQLPlanner
TableStatsCache *stats.TableStatisticsCache
StatsRefresher *stats.Refresher
ExecLogger *log.SecondaryLogger
AuditLogger *log.SecondaryLogger
SlowQueryLogger *log.SecondaryLogger
SlowInternalQueryLogger *log.SecondaryLogger
AuthLogger *log.SecondaryLogger
InternalExecutor *InternalExecutor
QueryCache *querycache.C

TestingKnobs ExecutorTestingKnobs
PGWireTestingKnobs *PGWireTestingKnobs
Expand Down

0 comments on commit 71d1502

Please sign in to comment.