Skip to content

Commit

Permalink
Merge #52377 #53029
Browse files Browse the repository at this point in the history
52377: sql: add a new optional secondary logger for internal slow queries r=solongordon a=arulajmani

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 adds
an optional cluster setting, `sql.log.slow_query.internal`. When set to
true, all internal queries that have service latency above the slow
query log threshold will be logged to
`cockroach-slow-log-internal-only.log` instead of
`cockroach-slow-log.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 instead of the main slow query log.

53029: Makefile: misc spacing adjustment for cleaner build output r=irfansharif a=irfansharif

Scratching a tiny itch. Before:

```
  docgen http \
                  --protoc ...
                  --gendoc ...
                  --out ...
                  --protobuf ...
```

After:

```
  docgen http \
          --protoc ...
          --gendoc ...
          --out ...
          --protobuf ...
```

Release note: None

Co-authored-by: arulajmani <arulajmani@gmail.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
3 people committed Aug 19, 2020
3 parents 281285d + 71d1502 + 878e7a8 commit f39153f
Show file tree
Hide file tree
Showing 5 changed files with 53 additions and 26 deletions.
8 changes: 4 additions & 4 deletions Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -1565,10 +1565,10 @@ bin/.docgen_functions: bin/docgen

bin/.docgen_http: bin/docgen $(PROTOC)
docgen http \
--protoc $(PROTOC) \
--gendoc ./bin/protoc-gen-doc \
--out docs/generated/http \
--protobuf pkg:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH):$(ERRORS_PATH)
--protoc $(PROTOC) \
--gendoc ./bin/protoc-gen-doc \
--out docs/generated/http \
--protobuf pkg:$(GOGO_PROTOBUF_PATH):$(PROTOBUF_PATH):$(COREOS_PATH):$(GRPC_GATEWAY_GOOGLEAPIS_PATH):$(ERRORS_PATH)
touch $@

.PHONY: docs/generated/redact_safe.md
Expand Down
1 change: 1 addition & 0 deletions docs/generated/settings/settings.html
Original file line number Diff line number Diff line change
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
Original file line number Diff line number Diff line change
Expand Up @@ -465,7 +465,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 @@ -479,20 +479,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 @@ -501,6 +504,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
Original file line number Diff line number Diff line change
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
Original file line number Diff line number Diff line change
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 f39153f

Please sign in to comment.