Skip to content

Commit

Permalink
sql,log: new structured event FirstQuery
Browse files Browse the repository at this point in the history
This change aims to expose “time to first query”, a product metric
that correlates with how easy it is for a user to start using
CockroachDB.

The naive approach to do this would be to store a boolean "has the
first query been reported in telemetry yet" inside a system table.
This naive approach is defective because it incurs a storage
read-eval-write cycle with an access to KV storage, or disk, or a
fan-out across the network, upon every query. This is an unacceptable
performance overhead.

Instead, we want the collection of this metric to have virtually no
performance impact on the common case of workloads beyond the first
query.

To get to a better design, we need to better understand the
requirement.  A naive interpretation of the requirement would be to
assume that we need the time to first query to be reported *exactly
once*.

Besides the fact that this is strictly impossible from a
computer-theoretical perspective [1], an exactly-once approach would
also incur unacceptable performance overhead, due to the need to
synchronize between multiple nodes, when the first query is sent
through a load balancer.

So what can we do instead?

In an initial implementation (#63812), we tried to *approximate*
“exactly once” reporting with a design that combines “at least once”
with a best effort of “at most once”. This was achieved using a
cluster setting. That solution was storing the time of the first query
inside a cluster setting, and relying on the auto-propagation of
setting changes through the cluster to announce to other nodes that
the first query had been reported already.

The use of a cluster setting also takes care of persisting this
information across node restarts, again towards the need to
approximate “at most once”.

After further discussion with stakeholders, it appears that there
is no strong requirement to reach “at most once”. The data processing
pipeline that consumes the "first query" event is only interested to
trigger the first time the event is recognized for a given cluster and
can be easily configured to ignore events reported after that.

The only practical consideration is to minimize the amount of
telemetry data (to ensure scalability over the CockroachCloud fleet),
and so we are not interested to send a "first query" event for *every
single query executed* either (obviously).

Based on this new understanding, this commit simplifies the approach
taken in #63812: it makes every CockroachDB *process* report the first
SQL query it executes on the TELEMETRY channel. Whether the process
has reported the event already or not is stored in a memory-only
atomic variable.

This approach technically has "at most once" semantics per node (an
event can get lost if the logging output is stalled at the moment the
event is reported), but approximates "at least once" semantics very
well for multi-node clusters, or when nodes restart over time—as,
presumably, the logging system will not be clogged all the time.

There are two open questions in this approach however:

- when a CockroachCloud cluster is initially configured by the CC
  intrusion control plane), there are “administrative” queries that
  are *not internal* but also *not user initiated*.

  With the current patch, the "first query" event will be reported
  for these administrative SQL queries. Is this desired?
  (NB: the same question applied to #63812)

  If not, we will need to agree on a reliable way to identify
  a user-initiated query.

  (A similar question exists when the user opens `cockroach sql`, but
  does not type anything in. The `cockroach sql` command sends queries
  in the background, and would trigger the "first query" event even
  without user interaction. Are we OK with this?)

- this approach will cause a "first query" event to be logged *at
  least once per SQL pod* (when multiple pods are running side-by-side),
  and also *once after every pod restart*.

  To retrieve the “time **to** first query” (i.e. not “time **of**
  first query”), the consumer of these events must then compute the
  minimum timestamp of all the first query events received, and
  substract the timestamp of cluster creation.  (The cluster creation
  timestamp can be obtained either via an external source, e.g. when
  the customer initiated the orchestration to create the cluster, or
  by retrieving the first ``node_restart`` event from the OPS logging
  channel.)

  Is this behavior acceptable?

Of note: departure from "at most once" delivery has another advantage
with regards to product telemetry: it teaches us something about the
effective utilization of all the nodes in a cluster. If we know the
cluster has 10 nodes but only 3 report "first query" events, that
might teach us that the cluster is over-provisioned.

[1] in a distributed system, an event can have guaranteed
at-least-once delivery, or at-most-once, but never both. We also have
the same constraint for CDC.

Release note (cli change): CockroachDB now reports a `first_query`
structured event on the TELEMETRY channel for the first SQL query
executed on each server process.
  • Loading branch information
knz committed Jun 14, 2021
1 parent 1295502 commit 955825c
Show file tree
Hide file tree
Showing 15 changed files with 1,108 additions and 598 deletions.
3 changes: 2 additions & 1 deletion Makefile
Original file line number Diff line number Diff line change
Expand Up @@ -1611,7 +1611,8 @@ EVENTLOG_PROTOS = \
pkg/util/log/eventpb/sql_audit_events.proto \
pkg/util/log/eventpb/cluster_events.proto \
pkg/util/log/eventpb/job_events.proto \
pkg/util/log/eventpb/health_events.proto
pkg/util/log/eventpb/health_events.proto \
pkg/util/log/eventpb/telemetry.proto

LOGSINKDOC_DEP = pkg/util/log/logconfig/config.go

Expand Down
38 changes: 38 additions & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -1992,6 +1992,44 @@ An event of type `drop_role` is recorded when a role is dropped.
| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends |
| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes |

## Telemetry events



Events in this category are logged to the `TELEMETRY` channel.


### `first_query`

An event of type `first_query` is reported every time a SQL server process runs a SQL query
on behalf of an external client for the first time.
(This excludes queries ran by internal executors.)




#### Common fields

| Field | Description | Sensitive |
|--|--|--|
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |
| `Statement` | A normalized copy of the SQL statement that triggered the event. | yes |
| `Tag` | The statement tag. This is separate from the statement string, since the statement string can contain sensitive information. The tag is guaranteed not to. | no |
| `User` | The user account that triggered the event. The special usernames `root` and `node` are not considered sensitive. | depends |
| `DescriptorID` | The primary object descriptor affected by the operation. Set to zero for operations that don't affect descriptors. | no |
| `ApplicationName` | The application name for the session where the event was emitted. This is included in the event to ease filtering of logging output by application. Application names starting with a dollar sign (`$`) are not considered sensitive. | depends |
| `PlaceholderValues` | The mapping of SQL placeholders to their values, for prepared statements. | yes |
| `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 |
| `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 |
| `FullIndexScan` | Whether the query contains a full secondary index scan. | no |
| `TxnCounter` | The sequence number of the SQL transaction inside its session. | no |

## Zone config events

Events in this category pertain to zone configuration changes on
Expand Down
4 changes: 2 additions & 2 deletions docs/generated/logging.md
Original file line number Diff line number Diff line change
Expand Up @@ -164,9 +164,9 @@ helping developers of CockroachDB itself. It exists as a separate
channel so as to not pollute the `SQL_PERF` logging output with
internal troubleshooting details.

## TELEMETRY
### `TELEMETRY`

The TELEMETRY channel reports telemetry events. Telemetry events describe
The `TELEMETRY` channel reports telemetry events. Telemetry events describe
feature usage within CockroachDB and anonymizes any application-
specific data.

17 changes: 16 additions & 1 deletion pkg/sql/exec_log.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,7 @@ import (
"bytes"
"context"
"fmt"
"sync/atomic"
"time"

"github.com/cockroachdb/cockroach/pkg/settings"
Expand Down Expand Up @@ -161,12 +162,21 @@ func (p *planner) maybeLogStatementInternal(
// If hasAdminRoleCache IsSet is true iff AdminAuditLog is enabled.
shouldLogToAdminAuditLog := hasAdminRoleCache.IsSet && hasAdminRoleCache.HasAdminRole

// Record the first query timestamp for non-internal statements, if
// not already known.
logFirstQuery := false
if isInternal := execType == executorTypeInternal; !isInternal {
if swapped := atomic.CompareAndSwapInt64(&p.execCfg.firstQueryTimestamp, 0, startTime.UnixNano()); swapped {
logFirstQuery = true
}
}

// Only log to adminAuditLog if the statement is executed by
// a user and the user has admin privilege (is directly or indirectly a
// member of the admin role).

if !logV && !logExecuteEnabled && !auditEventsDetected && !slowQueryLogEnabled &&
!shouldLogToAdminAuditLog {
!shouldLogToAdminAuditLog && !logFirstQuery {
// Shortcut: avoid the expense of computing anything log-related
// if logging is not enabled by configuration.
return
Expand Down Expand Up @@ -347,6 +357,11 @@ func (p *planner) maybeLogStatementInternal(
if shouldLogToAdminAuditLog {
p.logEventsOnlyExternally(ctx, eventLogEntry{event: &eventpb.AdminQuery{CommonSQLExecDetails: execDetails}})
}

if logFirstQuery {
p.logEventsOnlyExternally(ctx,
eventLogEntry{event: &eventpb.FirstQuery{CommonSQLExecDetails: execDetails}})
}
}

func (p *planner) logEventsOnlyExternally(ctx context.Context, entries ...eventLogEntry) {
Expand Down
12 changes: 12 additions & 0 deletions pkg/sql/exec_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -918,6 +918,18 @@ type ExecutorConfig struct {
// CompactEngineSpanFunc is used to inform a storage engine of the need to
// perform compaction over a key span.
CompactEngineSpanFunc tree.CompactEngineSpanFunc

// firstQueryTimestamp is the timestamp at which the process
// executed a non-internal query for the first time. This
// is used to gate the emission of a FirstQuery telemetry event.
// This is accessed atomically.
//
// TODO(knz): Find a better location for this variable.
// It needs to be common to all executor instances and so
// cannot be stored on the planner. However it is internal
// to the exec log machinery and therefore does not deserve
// to be owned by the SQL stats package.
firstQueryTimestamp int64
}

// VersionUpgradeHook is used to run migrations starting in v21.1.
Expand Down
2 changes: 1 addition & 1 deletion pkg/util/log/eventpb/eventlog_channels_generated.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

Loading

0 comments on commit 955825c

Please sign in to comment.