Skip to content

Commit

Permalink
log: add protobuf messages for telemetry txn events
Browse files Browse the repository at this point in the history
This commit adds the  messages listed below to `telemetry.proto` in
preparation for sending transaction executions to the telemetry
channel. The transaction event that is eventually sent should  contain
all execution information currently being tracked for transaction
fingerprints.

- `SampledTransaction`: contains fields equivalent to the execution
information stored by `CollectedTransactionStatistics` from
app_stats.proto, but represents a single txn execution instead
of aggregated executions of a transaction fingerprint.
- `SampledExecStats`: used as a field in `SampledTransaction`, it
contains execution stats that are sampled. This event is the equivalent
to `ExecStats` from app_stats.proto but for a single execution.
- `MVCCIteratorStats`: used in `SampledExecStats` above, the equivalent of
MVCCIteratorStats from app_stats.proto but for a single execution.

In addition, in order to support the above fields a couple of additional
code templates have been added for generating json log encoding:
- array_of_uint64 type is now being handled for json logs
- `nestedMessage` has been added as a custom type in `gen.go`. Object field
types can be assigned to this type in order to generate them as nested
objects.

Part of: #108284

Release note: None
  • Loading branch information
xinhaoz committed Jan 29, 2024
1 parent 7b6c2b7 commit 6e2b497
Show file tree
Hide file tree
Showing 8 changed files with 686 additions and 6 deletions.
84 changes: 84 additions & 0 deletions docs/generated/eventlog.md
Original file line number Diff line number Diff line change
Expand Up @@ -2879,6 +2879,29 @@ An event of type `hot_ranges_stats`
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |

### `m_v_c_c_iterator_stats`

Internal storage iteration statistics for a single execution.


| Field | Description | Sensitive |
|--|--|--|
| `StepCount` | StepCount collects the number of times the iterator moved forward or backward over the DB's underlying storage keyspace. For details, see pkg/storage/engine.go and pkg/sql/opt/exec/factory.go. | no |
| `StepCountInternal` | StepCountInternal collects the number of times the iterator moved forward or backward over LSM internal keys. For details, see pkg/storage/engine.go and pkg/sql/opt/exec/factory.go. | no |
| `SeekCount` | SeekCount collects the number of times the iterator moved to a specific key/value pair in the DB's underlying storage keyspace. For details, see pkg/storage/engine.go and pkg/sql/opt/exec/factory.go. | no |
| `SeekCountInternal` | SeekCountInternal collects the number of times the iterator moved to a specific LSM internal key. For details, see pkg/storage/engine.go and pkg/sql/opt/exec/factory.go. | no |
| `BlockBytes` | BlockBytes collects the bytes in the loaded SSTable data blocks. For details, see pebble.InternalIteratorStats. | no |
| `BlockBytesInCache` | BlockBytesInCache collects the subset of BlockBytes in the block cache. For details, see pebble.InternalIteratorStats. | no |
| `KeyBytes` | KeyBytes collects the bytes in keys that were iterated over. For details, see pebble.InternalIteratorStats. | no |
| `ValueBytes` | ValueBytes collects the bytes in values that were iterated over. For details, see pebble.InternalIteratorStats. | no |
| `PointCount` | PointCount collects the count of point keys iterated over. For details, see pebble.InternalIteratorStats. | no |
| `PointsCoveredByRangeTombstones` | PointsCoveredByRangeTombstones collects the count of point keys that were iterated over that were covered by range tombstones. For details, see pebble.InternalIteratorStats and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no |
| `RangeKeyCount` | RangeKeyCount collects the count of range keys encountered during iteration. For details, see pebble.RangeKeyIteratorStats and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no |
| `RangeKeyContainedPoints` | RangeKeyContainedPoints collects the count of point keys encountered within the bounds of a range key. For details, see pebble.RangeKeyIteratorStats and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no |
| `RangeKeySkippedPoints` | RangeKeySkippedPoints collects the count of the subset of ContainedPoints point keys that were skipped during iteration due to range-key masking. For details, see pkg/storage/engine.go, pebble.RangeKeyIteratorStats, and https://github.com/cockroachdb/cockroach/blob/master/docs/tech-notes/mvcc-range-tombstones.md. | no |



### `recovery_event`

An event of type `recovery_event` is an event that is logged on every invocation of BACKUP,
Expand Down Expand Up @@ -2924,6 +2947,27 @@ logged whenever a BACKUP and RESTORE job completes or fails.
| `Timestamp` | The timestamp of the event. Expressed as nanoseconds since the Unix epoch. | no |
| `EventType` | The type of the event. | no |

### `sampled_exec_stats`

An event of type `sampled_exec_stats` contains execution statistics that apply to both statements
and transactions. These stats as a whole are collected using a sampling approach.
These exec stats are meant to contain the same fields as ExecStats in
apps_stats.proto but are for a single execution rather than aggregated executions.
Fields in this struct should be updated in sync with apps_stats.proto.


| Field | Description | Sensitive |
|--|--|--|
| `NetworkBytes` | NetworkBytes collects the number of bytes sent over the network. | no |
| `MaxMemUsage` | MaxMemUsage collects the maximum memory usage that occurred on a node. | no |
| `ContentionTime` | ContentionTime collects the time in seconds statements in the transaction spent contending. | no |
| `NetworkMessages` | NetworkMessages collects the number of messages that were sent over the network. | no |
| `MaxDiskUsage` | MaxDiskUsage collects the maximum temporary disk usage that occurred. This is set in cases where a query had to spill to disk, e.g. when performing a large sort where not all of the tuples fit in memory. | no |
| `CPUSQLNanos` | CPUSQLNanos collects the CPU time spent executing SQL operations in nanoseconds. Currently, it is only collected for statements without mutations that have a vectorized plan. | no |
| `MVCCIteratorStats` | Internal storage iteration statistics. | yes |



### `sampled_query`

An event of type `sampled_query` is the SQL query event logged to the telemetry channel. It
Expand Down Expand Up @@ -3030,6 +3074,46 @@ contains common SQL event/execution details.
| `BulkJobId` | The job id for bulk job (IMPORT/BACKUP/RESTORE). | no |
| `StmtPosInTxn` | The statement's index in the transaction, starting at 1. | no |

### `sampled_transaction`

An event of type `sampled_transaction` is the event logged to telemetry at the end of transaction execution.


| Field | Description | Sensitive |
|--|--|--|
| `User` | User is the user account that triggered the transaction. The special usernames `root` and `node` are not considered sensitive. | depends |
| `ApplicationName` | ApplicationName is the application name for the session where the transaction was executed. This is included in the event to ease filtering of logging output by application. | no |
| `TxnCounter` | TxnCounter is the sequence number of the SQL transaction inside its session. | no |
| `SessionID` | SessionID is the ID of the session that initiated the transaction. | no |
| `TransactionID` | TransactionID is the id of the transaction. | no |
| `TransactionFingerprintID` | TransactionFingerprintID is the fingerprint ID of the transaction. This can be used to find the transaction in the console. | no |
| `Committed` | Committed indicates if the transaction committed successfully. We want to include this value even if it is false. | no |
| `ImplicitTxn` | ImplicitTxn indicates if the transaction was an implicit one. We want to include this value even if it is false. | no |
| `StartTimeUnixNanos` | StartTimeUnixNanos is the time the transaction was started. Expressed as unix time in nanoseconds. | no |
| `EndTimeUnixNanos` | EndTimeUnixNanos the time the transaction finished (either committed or aborted). Expressed as unix time in nanoseconds. | no |
| `ServiceLatNanos` | ServiceLatNanos is the time to service the whole transaction, from start to end of execution. | no |
| `SQLSTATE` | SQLSTATE is the SQLSTATE code for the error, if an error was encountered. Empty/omitted if no error. | no |
| `ErrorText` | ErrorText is the text of the error if any. | partially |
| `NumRetries` | NumRetries is the number of time when the txn was retried automatically by the server. | no |
| `LastAutoRetryReason` | LastAutoRetryReason is a string containing the reason for the last automatic retry. | partially |
| `StatementFingerprintIDs` | StatementFingerprintIDs is an array of statement fingerprint IDs belonging to this transaction. | yes |
| `NumRows` | NumRows is the total number of rows returned across all statements. | no |
| `RetryLatNanos` | RetryLatNanos is the amount of time spent retrying the transaction. | no |
| `CommitLatNanos` | CommitLatNanos is the amount of time spent committing the transaction after all statement operations. | no |
| `IdleLatNanos` | IdleLatNanos is the amount of time spent waiting for the client to send statements while the transaction is open. | no |
| `BytesRead` | BytesRead is the number of bytes read from disk. | no |
| `RowsRead` | RowsRead is the number of rows read from disk. | no |
| `RowsWritten` | RowsWritten is the number of rows written to disk. | no |
| `SampledExecStats` | SampledExecStats is a nested field containing execution statistics. This field will be omitted if the stats were not sampled. | yes |


#### 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 |

### `schema_descriptor`

An event of type `schema_descriptor` is an event for schema telemetry, whose purpose is
Expand Down
3 changes: 3 additions & 0 deletions pkg/sql/appstatspb/app_stats.proto
Original file line number Diff line number Diff line change
Expand Up @@ -313,6 +313,9 @@ message TxnStats {
// ExecStats contains execution statistics that apply to both statements
// and transactions. These stats are currently collected using a sampling
// approach.
// When adding additional fields to this message, please make the corresponding
// changes to 'SampledExecStats' in telemetry.proto to keep the two messages in sync
// with respect to the information stored.
message ExecStats {
// Count keeps track of how many times execution stats were recorded. This is
// not necessarily equal to the number of times a statement/transaction was
Expand Down
1 change: 1 addition & 0 deletions pkg/util/log/eventpb/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ go_library(
importpath = "github.com/cockroachdb/cockroach/pkg/util/log/eventpb",
visibility = ["//visibility:public"],
deps = [
"//pkg/sql/appstatspb", # keep
"//pkg/util/jsonbytes", # keep
"//pkg/util/log/logpb",
"@com_github_cockroachdb_errors//:errors",
Expand Down
9 changes: 9 additions & 0 deletions 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.

12 changes: 6 additions & 6 deletions pkg/util/log/eventpb/eventpbgen/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -15,12 +15,6 @@ go_library(
],
)

go_binary(
name = "eventpbgen",
embed = [":eventpbgen_lib"],
visibility = ["//visibility:public"],
)

# The code generator

genrule(
Expand All @@ -38,3 +32,9 @@ genrule(
"//pkg/gen:__pkg__",
],
)

go_binary(
name = "eventpbgen",
embed = [":eventpbgen_lib"],
visibility = ["//visibility:public"],
)
24 changes: 24 additions & 0 deletions pkg/util/log/eventpb/eventpbgen/gen.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,12 @@ func readInput(
typ = "timestamp"
case "cockroach.sql.sqlbase.Descriptor":
typ = "protobuf"
case "MVCCIteratorStats":
fallthrough
case "SampledExecStats":
// This is necessary so that the fields in the
// message doesn't get inlined.
typ = "nestedMessage"
}

if otherMsg, ok := infos[typ]; ok {
Expand Down Expand Up @@ -660,6 +666,16 @@ func (m *{{.GoType}}) AppendJSONFields(printComma bool, b redact.RedactableBytes
}
b = append(b, ']')
}
{{- else if eq .FieldType "array_of_uint64" -}}
if len(m.{{.FieldName}}) > 0 {
if printComma { b = append(b, ',')}; printComma = true
b = append(b, "\"{{.FieldName}}\":["...)
for i, v := range m.{{.FieldName}} {
if i > 0 { b = append(b, ',') }
b = strconv.AppendUint(b, uint64(v), 10)
}
b = append(b, ']')
}
{{- else if .IsEnum }}
{{ if not .AllowZeroValue -}}
if m.{{.FieldName}} != 0 {
Expand Down Expand Up @@ -695,6 +711,14 @@ func (m *{{.GoType}}) AppendJSONFields(printComma bool, b redact.RedactableBytes
b = append(b, []byte(str)...)
}
}
{{- else if eq .FieldType "nestedMessage"}}
if m.{{.FieldName}} != nil {
if printComma { b = append(b, ',')}; printComma = true
b = append(b, "\"{{.FieldName}}\":"...)
b = append(b, '{')
printComma, b = m.{{.FieldName}}.AppendJSONFields(false, b)
b = append(b, '}')
}
{{- else}}
{{ error .FieldType }}
{{- end}}
Expand Down
Loading

0 comments on commit 6e2b497

Please sign in to comment.