-
Notifications
You must be signed in to change notification settings - Fork 109
Expand file tree
/
Copy pathadapter_stats.rs
More file actions
369 lines (350 loc) · 14.8 KB
/
adapter_stats.rs
File metadata and controls
369 lines (350 loc) · 14.8 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
use bytemuck::NoUninit;
use chrono::{DateTime, Utc};
use serde::{Deserialize, Serialize};
use serde_json::Value as JsonValue;
use std::collections::BTreeMap;
use utoipa::ToSchema;
use uuid::Uuid;
use crate::{
coordination::Step,
suspend::SuspendError,
transaction::{CommitProgressSummary, TransactionId},
};
/// Pipeline state.
#[derive(Copy, Clone, Debug, Default, PartialEq, Eq, Serialize, Deserialize, ToSchema)]
#[serde(rename_all = "PascalCase")]
pub enum PipelineState {
/// All input endpoints are paused (or are in the process of being paused).
#[default]
Paused,
/// Controller is running.
Running,
/// Controller is being terminated.
Terminated,
}
// Metrics for an input endpoint.
///
/// Serializes to match the subset of fields needed for error tracking.
#[derive(Default, Serialize, Deserialize, Debug, Clone)]
pub struct InputEndpointErrorMetrics {
pub endpoint_name: String,
#[serde(default)]
pub num_transport_errors: u64,
#[serde(default)]
pub num_parse_errors: u64,
}
/// Metrics for an output endpoint.
///
/// Serializes to match the subset of fields needed for error tracking.
#[derive(Default, Serialize, Deserialize, Debug, Clone)]
pub struct OutputEndpointErrorMetrics {
pub endpoint_name: String,
#[serde(default)]
pub num_encode_errors: u64,
#[serde(default)]
pub num_transport_errors: u64,
}
/// Endpoint statistics containing metrics.
///
/// Wraps metrics in a structure that matches the full stats API shape.
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct EndpointErrorStats<T> {
#[serde(default)]
pub metrics: T,
}
/// Schema definition for endpoint config that only includes the stream field.
#[derive(Debug, Deserialize, Serialize, ToSchema)]
pub struct ShortEndpointConfig {
/// The name of the stream.
pub stream: String,
}
/// Pipeline error statistics response from the runtime.
///
/// Lightweight response containing only error counts from all endpoints.
#[derive(Serialize, Deserialize, Debug, Clone)]
pub struct PipelineStatsErrorsResponse {
#[serde(default)]
pub inputs: Vec<EndpointErrorStats<InputEndpointErrorMetrics>>,
#[serde(default)]
pub outputs: Vec<EndpointErrorStats<OutputEndpointErrorMetrics>>,
}
// OpenAPI schema definitions for controller statistics
// These match the serialized JSON structure from the adapters crate
/// Transaction status summarized as a single value.
#[derive(
Debug, Default, Copy, PartialEq, Eq, Clone, NoUninit, Serialize, Deserialize, ToSchema,
)]
#[repr(u8)]
pub enum TransactionStatus {
#[default]
NoTransaction,
TransactionInProgress,
CommitInProgress,
}
/// Transaction phase.
#[derive(Clone, Copy, PartialEq, Eq, Debug, Deserialize, Serialize, ToSchema)]
#[serde(rename_all = "PascalCase")]
#[schema(as = TransactionPhase)]
pub enum ExternalTransactionPhase {
/// Transaction is in progress.
Started,
/// Transaction has been committed.
Committed,
}
/// Connector transaction phase with debugging label.
#[derive(Clone, PartialEq, Eq, Debug, Deserialize, Serialize, ToSchema)]
#[schema(as = ConnectorTransactionPhase)]
pub struct ExternalConnectorTransactionPhase {
/// Current phase of the transaction.
#[schema(value_type = TransactionPhase)]
pub phase: ExternalTransactionPhase,
/// Optional label for debugging.
pub label: Option<String>,
}
/// Information about entities that initiated the current transaction.
#[derive(Clone, Default, Debug, Deserialize, Serialize, ToSchema)]
#[schema(as = TransactionInitiators)]
pub struct ExternalTransactionInitiators {
/// ID assigned to the transaction (None if no transaction is in progress).
#[schema(value_type = Option<i64>)]
pub transaction_id: Option<TransactionId>,
/// Transaction phase initiated by the API.
#[schema(value_type = Option<TransactionPhase>)]
pub initiated_by_api: Option<ExternalTransactionPhase>,
/// Transaction phases initiated by connectors, indexed by endpoint name.
#[schema(value_type = BTreeMap<String, ConnectorTransactionPhase>)]
pub initiated_by_connectors: BTreeMap<String, ExternalConnectorTransactionPhase>,
}
/// A watermark that has been fully processed by the pipeline.
#[derive(Clone, Debug, Deserialize, Serialize, ToSchema)]
#[schema(as = CompletedWatermark)]
pub struct ExternalCompletedWatermark {
/// Metadata that describes the position in the input stream (e.g., Kafka partition/offset pairs).
#[schema(value_type = Object)]
pub metadata: JsonValue,
/// Timestamp when the data was ingested from the wire.
pub ingested_at: String,
/// Timestamp when the data was processed by the circuit.
pub processed_at: String,
/// Timestamp when all outputs produced from this input have been pushed to all output endpoints.
pub completed_at: String,
}
/// Performance metrics for an input endpoint.
#[derive(Debug, Default, Deserialize, Serialize, ToSchema)]
#[schema(as = InputEndpointMetrics)]
pub struct ExternalInputEndpointMetrics {
/// Total bytes pushed to the endpoint since it was created.
pub total_bytes: u64,
/// Total records pushed to the endpoint since it was created.
pub total_records: u64,
/// Number of records currently buffered by the endpoint (not yet consumed by the circuit).
pub buffered_records: u64,
/// Number of bytes currently buffered by the endpoint (not yet consumed by the circuit).
pub buffered_bytes: u64,
/// Number of transport errors.
pub num_transport_errors: u64,
/// Number of parse errors.
pub num_parse_errors: u64,
/// True if end-of-input has been signaled.
pub end_of_input: bool,
}
/// Input endpoint status information.
#[derive(Debug, Serialize, Deserialize, ToSchema)]
#[schema(as = InputEndpointStatus)]
pub struct ExternalInputEndpointStatus {
/// Endpoint name.
pub endpoint_name: String,
/// Endpoint configuration.
pub config: ShortEndpointConfig,
/// Performance metrics.
#[schema(value_type = InputEndpointMetrics)]
pub metrics: ExternalInputEndpointMetrics,
/// The first fatal error that occurred at the endpoint.
pub fatal_error: Option<String>,
/// Endpoint has been paused by the user.
pub paused: bool,
/// Endpoint is currently a barrier to checkpointing and suspend.
pub barrier: bool,
/// The latest completed watermark.
#[schema(value_type = Option<CompletedWatermark>)]
pub completed_frontier: Option<ExternalCompletedWatermark>,
}
/// Performance metrics for an output endpoint.
#[derive(Debug, Default, Deserialize, Serialize, ToSchema, PartialEq, Eq, PartialOrd, Ord)]
#[schema(as = OutputEndpointMetrics)]
pub struct ExternalOutputEndpointMetrics {
/// Records sent on the underlying transport.
pub transmitted_records: u64,
/// Bytes sent on the underlying transport.
pub transmitted_bytes: u64,
/// Number of queued records.
pub queued_records: u64,
/// Number of queued batches.
pub queued_batches: u64,
/// Number of records pushed to the output buffer.
pub buffered_records: u64,
/// Number of batches in the buffer.
pub buffered_batches: u64,
/// Number of encoding errors.
pub num_encode_errors: u64,
/// Number of transport errors.
pub num_transport_errors: u64,
/// The number of input records processed by the circuit.
///
/// This metric tracks the end-to-end progress of the pipeline: the output
/// of this endpoint is equal to the output of the circuit after
/// processing `total_processed_input_records` records.
///
/// In a multihost pipeline, this count reflects only the input records
/// processed on the same host as the output endpoint, which is not usually
/// meaningful.
pub total_processed_input_records: u64,
/// The number of steps whose input records have been processed by the
/// endpoint.
///
/// This is meaningful in a multihost pipeline because steps are
/// synchronized across all of the hosts.
///
/// # Interpretation
///
/// This is a count, not a step number. If `total_processed_steps` is 0, no
/// steps have been processed to completion. If `total_processed_steps >
/// 0`, then the last step whose input records have been processed to
/// completion is `total_processed_steps - 1`. A record that was ingested in
/// step `n` is fully processed when `total_processed_steps > n`.
#[schema(value_type = u64)]
pub total_processed_steps: Step,
/// Extra memory in use beyond that used for queuing records.
pub memory: u64,
}
/// Output endpoint status information.
#[derive(Debug, Deserialize, Serialize, ToSchema)]
#[schema(as = OutputEndpointStatus)]
pub struct ExternalOutputEndpointStatus {
/// Endpoint name.
pub endpoint_name: String,
/// Endpoint configuration.
pub config: ShortEndpointConfig,
/// Performance metrics.
#[schema(value_type = OutputEndpointMetrics)]
pub metrics: ExternalOutputEndpointMetrics,
/// The first fatal error that occurred at the endpoint.
pub fatal_error: Option<String>,
}
/// Global controller metrics.
#[derive(Debug, Default, Serialize, Deserialize, ToSchema)]
#[schema(as = GlobalControllerMetrics)]
pub struct ExternalGlobalControllerMetrics {
/// State of the pipeline: running, paused, or terminating.
pub state: PipelineState,
/// The pipeline has been resumed from a checkpoint and is currently bootstrapping new and modified views.
pub bootstrap_in_progress: bool,
/// Status of the current transaction.
pub transaction_status: TransactionStatus,
/// ID of the current transaction or 0 if no transaction is in progress.
#[schema(value_type = i64)]
pub transaction_id: TransactionId,
/// Progress of the current transaction commit, if one is in progress.
pub commit_progress: Option<CommitProgressSummary>,
/// Entities that initiated the current transaction.
#[schema(value_type = TransactionInitiators)]
pub transaction_initiators: ExternalTransactionInitiators,
/// Resident set size of the pipeline process, in bytes.
pub rss_bytes: u64,
/// CPU time used by the pipeline across all threads, in milliseconds.
pub cpu_msecs: u64,
/// Time since the pipeline process started, including time that the
/// pipeline was running or paused.
///
/// This is the elapsed time since `start_time`.
pub uptime_msecs: u64,
/// Time at which the pipeline process started, in seconds since the epoch.
#[serde(with = "chrono::serde::ts_seconds")]
#[schema(value_type = u64)]
pub start_time: DateTime<Utc>,
/// Uniquely identifies the pipeline process that started at start_time.
pub incarnation_uuid: Uuid,
/// Time at which the pipeline process from which we resumed started, in seconds since the epoch.
#[serde(with = "chrono::serde::ts_seconds")]
#[schema(value_type = u64)]
pub initial_start_time: DateTime<Utc>,
/// Current storage usage in bytes.
pub storage_bytes: u64,
/// Storage usage integrated over time, in megabytes * seconds.
pub storage_mb_secs: u64,
/// Time elapsed while the pipeline is executing a step, multiplied by the number of threads, in milliseconds.
pub runtime_elapsed_msecs: u64,
/// Total number of records currently buffered by all endpoints.
pub buffered_input_records: u64,
/// Total number of bytes currently buffered by all endpoints.
pub buffered_input_bytes: u64,
/// Total number of records received from all endpoints.
pub total_input_records: u64,
/// Total number of bytes received from all endpoints.
pub total_input_bytes: u64,
/// Total number of input records processed by the DBSP engine.
pub total_processed_records: u64,
/// Total bytes of input records processed by the DBSP engine.
pub total_processed_bytes: u64,
/// Total number of input records processed to completion.
pub total_completed_records: u64,
/// If the pipeline is stalled because one or more output connectors' output
/// buffers are full, this is the number of milliseconds that the current
/// stall has lasted.
///
/// If this is nonzero, then the output connectors causing the stall can be
/// identified by noticing `ExternalOutputEndpointMetrics::queued_records`
/// is greater than or equal to `ConnectorConfig::max_queued_records`.
///
/// In the ordinary case, the pipeline is not stalled, and this value is 0.
pub output_stall_msecs: u64,
/// Number of steps that have been initiated.
///
/// # Interpretation
///
/// This is a count, not a step number. If `total_initiated_steps` is 0, no
/// steps have been initiated. If `total_initiated_steps > 0`, then step
/// `total_initiated_steps - 1` has been started and all steps previous to
/// that have been completely processed by the circuit.
#[schema(value_type = u64)]
pub total_initiated_steps: Step,
/// Number of steps whose input records have been processed to completion.
///
/// A record is processed to completion if it has been processed by the DBSP engine and
/// all outputs derived from it have been processed by all output connectors.
///
/// # Interpretation
///
/// This is a count, not a step number. If `total_completed_steps` is 0, no
/// steps have been processed to completion. If `total_completed_steps >
/// 0`, then the last step whose input records have been processed to
/// completion is `total_completed_steps - 1`. A record that was ingested
/// when `total_initiated_steps` was `n` is fully processed when
/// `total_completed_steps >= n`.
#[schema(value_type = u64)]
pub total_completed_steps: Step,
/// True if the pipeline has processed all input data to completion.
pub pipeline_complete: bool,
}
/// Complete pipeline statistics returned by the `/stats` endpoint.
///
/// This schema definition matches the serialized JSON structure from
/// `adapters::controller::ControllerStatus`. The actual implementation with
/// atomics and mutexes lives in the adapters crate, which uses ExternalControllerStatus to
/// register this OpenAPI schema, making it available to pipeline-manager
/// without requiring a direct dependency on the adapters crate.
#[derive(Debug, Deserialize, Serialize, ToSchema, Default)]
#[schema(as = ControllerStatus)]
pub struct ExternalControllerStatus {
/// Global controller metrics.
#[schema(value_type = GlobalControllerMetrics)]
pub global_metrics: ExternalGlobalControllerMetrics,
/// Reason why the pipeline cannot be suspended or checkpointed (if any).
pub suspend_error: Option<SuspendError>,
/// Input endpoint configs and metrics.
#[schema(value_type = Vec<InputEndpointStatus>)]
pub inputs: Vec<ExternalInputEndpointStatus>,
/// Output endpoint configs and metrics.
#[schema(value_type = Vec<OutputEndpointStatus>)]
pub outputs: Vec<ExternalOutputEndpointStatus>,
}