micromegas-analytics 0.23.0

analytics module of micromegas
Documentation
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
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
use super::{
    lakehouse_context::LakehouseContext, partition_cache::QueryPartitionProvider,
    process_streams::get_process_thread_list, session_configurator::NoOpSessionConfigurator,
    view_factory::ViewFactory,
};
use crate::dfext::{
    string_column_accessor::string_column_by_name, typed_column::typed_column_by_name,
};
use crate::time::TimeRange;
use async_stream::stream;
use datafusion::{
    arrow::{
        array::{RecordBatch, TimestampNanosecondArray, UInt32Array},
        datatypes::SchemaRef,
    },
    catalog::{Session, TableProvider},
    common::Result as DFResult,
    execution::{SendableRecordBatchStream, TaskContext},
    logical_expr::{Expr, TableType},
    physical_expr::EquivalenceProperties,
    physical_plan::{
        DisplayAs, DisplayFormatType, ExecutionPlan, Partitioning, PlanProperties,
        execution_plan::{Boundedness, EmissionType},
        limit::GlobalLimitExec,
        stream::RecordBatchStreamAdapter,
    },
};
use futures::{StreamExt, TryStreamExt, stream};
use micromegas_perfetto::{chunk_sender::ChunkSender, streaming_writer::PerfettoWriter};
use micromegas_tracing::prelude::*;
use std::{
    any::Any,
    fmt::{self, Debug, Formatter},
    sync::Arc,
};

pub use super::process_spans_table_function::SpanTypes;

/// Execution plan that generates Perfetto trace chunks
pub struct PerfettoTraceExecutionPlan {
    schema: SchemaRef,
    process_id: String,
    span_types: SpanTypes,
    time_range: TimeRange,
    lakehouse: Arc<LakehouseContext>,
    view_factory: Arc<ViewFactory>,
    part_provider: Arc<dyn QueryPartitionProvider>,
    properties: PlanProperties,
}

impl PerfettoTraceExecutionPlan {
    pub fn new(
        schema: SchemaRef,
        process_id: String,
        span_types: SpanTypes,
        time_range: TimeRange,
        lakehouse: Arc<LakehouseContext>,
        view_factory: Arc<ViewFactory>,
        part_provider: Arc<dyn QueryPartitionProvider>,
    ) -> Self {
        let properties = PlanProperties::new(
            EquivalenceProperties::new(schema.clone()),
            Partitioning::UnknownPartitioning(1),
            EmissionType::Final,
            Boundedness::Bounded,
        );

        Self {
            schema,
            process_id,
            span_types,
            time_range,
            lakehouse,
            view_factory,
            part_provider,
            properties,
        }
    }
}

impl Debug for PerfettoTraceExecutionPlan {
    fn fmt(&self, f: &mut Formatter<'_>) -> fmt::Result {
        f.debug_struct("PerfettoTraceExecutionPlan")
            .field("process_id", &self.process_id)
            .field("span_types", &self.span_types)
            .field("time_range", &self.time_range)
            .finish()
    }
}

impl DisplayAs for PerfettoTraceExecutionPlan {
    fn fmt_as(&self, _t: DisplayFormatType, f: &mut fmt::Formatter) -> fmt::Result {
        write!(
            f,
            "PerfettoTraceExecutionPlan: process_id={}, span_types={:?}, time_range={}..{}",
            self.process_id, self.span_types, self.time_range.begin, self.time_range.end
        )
    }
}

impl ExecutionPlan for PerfettoTraceExecutionPlan {
    fn name(&self) -> &str {
        "PerfettoTraceExecutionPlan"
    }

    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        self.schema.clone()
    }

    fn properties(&self) -> &PlanProperties {
        &self.properties
    }

    fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>> {
        vec![]
    }

    fn with_new_children(
        self: Arc<Self>,
        _children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> DFResult<Arc<dyn ExecutionPlan>> {
        Ok(self)
    }

    #[span_fn]
    fn execute(
        &self,
        _partition: usize,
        _context: Arc<TaskContext>,
    ) -> DFResult<SendableRecordBatchStream> {
        let schema = self.schema.clone();
        let process_id = self.process_id.clone();
        let span_types = self.span_types;
        let time_range = self.time_range;
        let lakehouse = self.lakehouse.clone();
        let view_factory = self.view_factory.clone();
        let part_provider = self.part_provider.clone();

        // Create the stream directly without channels
        let stream = generate_perfetto_trace_stream(
            process_id,
            span_types,
            time_range,
            lakehouse,
            view_factory,
            part_provider,
        );

        Ok(Box::pin(RecordBatchStreamAdapter::new(schema, stream)))
    }
}

/// Creates a stream of Perfetto trace chunks using streaming architecture
#[span_fn]
fn generate_perfetto_trace_stream(
    process_id: String,
    span_types: SpanTypes,
    time_range: TimeRange,
    lakehouse: Arc<LakehouseContext>,
    view_factory: Arc<ViewFactory>,
    part_provider: Arc<dyn QueryPartitionProvider>,
) -> impl futures::Stream<Item = DFResult<RecordBatch>> {
    stream! {
        // Create channel for streaming chunks
        const CHUNK_SIZE: usize = 8 * 1024; // 8KB chunks
        let (chunk_sender, mut chunk_receiver) = tokio::sync::mpsc::channel(16);

        // Create ChunkSender that will stream data through the channel
        let chunk_sender_writer = ChunkSender::new(chunk_sender, CHUNK_SIZE);

        // Spawn background task to generate trace
        let generation_task = spawn_with_context(async move {
            generate_streaming_perfetto_trace(
                chunk_sender_writer,
                process_id,
                span_types,
                time_range,
                lakehouse,
                view_factory,
                part_provider,
            ).await
        });

        // Stream chunks as they become available
        while let Some(chunk_result) = chunk_receiver.recv().await {
            match chunk_result {
                Ok(batch) => yield Ok(batch),
                Err(e) => {
                    error!("Error in chunk generation: {:?}", e);
                    yield Err(datafusion::error::DataFusionError::Execution(
                        format!("Chunk generation failed: {}", e)
                    ));
                    return;
                }
            }
        }

        // Wait for generation task to complete and check for errors
        match generation_task.await {
            Ok(Ok(())) => {}, // Success
            Ok(Err(e)) => {
                error!("Trace generation failed: {:?}", e);
                yield Err(datafusion::error::DataFusionError::Execution(
                    format!("Trace generation failed: {}", e)
                ));
            }
            Err(e) => {
                error!("Task panicked: {:?}", e);
                yield Err(datafusion::error::DataFusionError::Execution(
                    format!("Task panicked: {}", e)
                ));
            }
        }
    }
}

/// Generate Perfetto trace using streaming architecture
async fn generate_streaming_perfetto_trace(
    chunk_sender: ChunkSender,
    process_id: String,
    span_types: SpanTypes,
    time_range: TimeRange,
    lakehouse: Arc<LakehouseContext>,
    view_factory: Arc<ViewFactory>,
    part_provider: Arc<dyn QueryPartitionProvider>,
) -> anyhow::Result<()> {
    info!(
        "Generating streaming Perfetto trace for process {} with span types {:?} from {} to {}",
        process_id, span_types, time_range.begin, time_range.end
    );

    // Create a context for making queries
    let ctx = super::query::make_session_context(
        lakehouse,
        part_provider,
        Some(TimeRange {
            begin: time_range.begin,
            end: time_range.end,
        }),
        view_factory,
        Arc::new(NoOpSessionConfigurator),
    )
    .await?;

    // Use ChunkSender directly as the writer destination
    let mut writer = PerfettoWriter::new(Box::new(chunk_sender), &process_id);

    let process_exe = get_process_exe(&process_id, &ctx).await?;
    writer.emit_process_descriptor(&process_exe).await?;
    writer.flush().await?; // Forces chunk emission

    let threads = get_process_thread_list(&process_id, &ctx).await?;
    for (stream_id, thread_id, thread_name) in &threads {
        writer
            .emit_thread_descriptor(stream_id, *thread_id, thread_name)
            .await?;
    }
    if !threads.is_empty() {
        writer.flush().await?; // Forces chunk emission
    }

    if matches!(span_types, SpanTypes::Async | SpanTypes::Both) {
        writer.emit_async_track_descriptor().await?;
        writer.flush().await?; // Forces chunk emission
    }

    if matches!(span_types, SpanTypes::Thread | SpanTypes::Both) {
        generate_thread_spans_with_writer(&mut writer, &ctx, &time_range, &threads).await?;
    }

    if matches!(span_types, SpanTypes::Async | SpanTypes::Both) {
        generate_async_spans_with_writer(&mut writer, &process_id, &ctx, &time_range).await?;
    }

    writer.flush().await?; // Final chunk - this handles the chunk_sender.flush() internally
    Ok(())
}

/// Get process executable name from the processes table
async fn get_process_exe(
    process_id: &str,
    ctx: &datafusion::execution::context::SessionContext,
) -> anyhow::Result<String> {
    let sql = format!(
        r#"
        SELECT exe
        FROM processes
        WHERE process_id = '{}'
        LIMIT 1
        "#,
        process_id
    );

    let df = ctx.sql(&sql).await?;
    let batches = df.collect().await?;

    if batches.is_empty() || batches[0].num_rows() == 0 {
        anyhow::bail!("Process {} not found", process_id);
    }

    let exes = string_column_by_name(&batches[0], "exe")?;
    Ok(exes.value(0)?.to_owned())
}

/// Format the SQL query for thread spans
fn format_thread_spans_query(stream_id: &str, time_range: &TimeRange) -> String {
    format!(
        r#"
        SELECT "begin", "end", name, filename, target, line
        FROM view_instance('thread_spans', '{}')
        WHERE begin <= TIMESTAMP '{}'
          AND end >= TIMESTAMP '{}'
        ORDER BY begin
        "#,
        stream_id,
        time_range.end.to_rfc3339(),
        time_range.begin.to_rfc3339()
    )
}

/// Generate thread spans with parallel JIT and sequential writing.
///
/// JIT partition locking is per-(view_set_name, view_instance_id), and each thread
/// has a unique stream_id used as view_instance_id. This means different threads
/// get different lock keys, making parallel JIT safe.
///
/// Strategy:
/// - Spawn tasks with spawn_with_context() for true multi-threaded parallelism
/// - Use buffered() to limit concurrent spawned tasks
/// - Collect all streams preserving order
/// - Consume streams sequentially to write each thread's spans together
async fn generate_thread_spans_with_writer(
    writer: &mut PerfettoWriter,
    ctx: &datafusion::execution::context::SessionContext,
    time_range: &TimeRange,
    threads: &[(String, i32, String)],
) -> anyhow::Result<()> {
    let max_concurrent = std::thread::available_parallelism()
        .map(|n| n.get())
        .unwrap_or(4);

    // Prepare query inputs upfront
    let queries: Vec<(String, String)> = threads
        .iter()
        .map(|(stream_id, _, _)| {
            (
                stream_id.clone(),
                format_thread_spans_query(stream_id, time_range),
            )
        })
        .collect();

    // Build streams in parallel using spawn for true multi-threading
    let streams: Vec<(String, SendableRecordBatchStream)> = stream::iter(queries)
        .map(|(stream_id, sql)| {
            let ctx = ctx.clone();
            async move {
                spawn_with_context(async move {
                    let df = ctx.sql(&sql).await?;
                    let stream = df.execute_stream().await?;
                    Ok::<_, anyhow::Error>((stream_id, stream))
                })
                .await?
            }
        })
        .buffered(max_concurrent)
        .try_collect()
        .await?;

    // Consume streams sequentially - each thread's spans written together
    for (stream_id, mut data_stream) in streams {
        writer.set_current_thread(&stream_id);

        let mut span_count = 0;
        while let Some(batch_result) = data_stream.next().await {
            let batch = batch_result?;
            let begin_times: &TimestampNanosecondArray = typed_column_by_name(&batch, "begin")?;
            let end_times: &TimestampNanosecondArray = typed_column_by_name(&batch, "end")?;
            let names = string_column_by_name(&batch, "name")?;
            let filenames = string_column_by_name(&batch, "filename")?;
            let targets = string_column_by_name(&batch, "target")?;
            let lines: &UInt32Array = typed_column_by_name(&batch, "line")?;

            for i in 0..batch.num_rows() {
                let begin_ns = begin_times.value(i) as u64;
                let end_ns = end_times.value(i) as u64;
                let name = names.value(i)?;
                let filename = filenames.value(i)?;
                let target = targets.value(i)?;
                let line = lines.value(i);

                writer
                    .emit_span(begin_ns, end_ns, name, target, filename, line)
                    .await?;

                span_count += 1;
                if span_count % 10 == 0 {
                    writer.flush().await?;
                }
            }
        }
    }
    Ok(())
}

/// Generate async spans using the provided PerfettoWriter
async fn generate_async_spans_with_writer(
    writer: &mut PerfettoWriter,
    process_id: &str,
    ctx: &datafusion::execution::context::SessionContext,
    time_range: &TimeRange,
) -> anyhow::Result<()> {
    let sql = format!(
        r#"
        WITH begin_events AS (
            SELECT span_id, time as begin_time, name, filename, target, line
            FROM view_instance('async_events', '{}')
            WHERE time >= TIMESTAMP '{}'
              AND time <= TIMESTAMP '{}'
              AND event_type = 'begin'
        ),
        end_events AS (
            SELECT span_id, time as end_time
            FROM view_instance('async_events', '{}')
            WHERE time >= TIMESTAMP '{}'
              AND time <= TIMESTAMP '{}'
              AND event_type = 'end'
        )
        SELECT 
            b.span_id,
            b.begin_time,
            e.end_time,
            b.name,
            b.filename,
            b.target,
            b.line
        FROM begin_events b
        INNER JOIN end_events e ON b.span_id = e.span_id
        ORDER BY b.begin_time
        "#,
        process_id,
        time_range.begin.to_rfc3339(),
        time_range.end.to_rfc3339(),
        process_id,
        time_range.begin.to_rfc3339(),
        time_range.end.to_rfc3339(),
    );

    let df = ctx.sql(&sql).await?;
    let mut stream = df.execute_stream().await?;

    let mut span_count = 0;
    while let Some(batch_result) = stream.next().await {
        let batch = batch_result?;
        let span_ids: &datafusion::arrow::array::Int64Array =
            typed_column_by_name(&batch, "span_id")?;
        let begin_times: &TimestampNanosecondArray = typed_column_by_name(&batch, "begin_time")?;
        let end_times: &TimestampNanosecondArray = typed_column_by_name(&batch, "end_time")?;
        let names = string_column_by_name(&batch, "name")?;
        let filenames = string_column_by_name(&batch, "filename")?;
        let targets = string_column_by_name(&batch, "target")?;
        let lines: &UInt32Array = typed_column_by_name(&batch, "line")?;
        for i in 0..batch.num_rows() {
            let _span_id = span_ids.value(i);
            let begin_ns = begin_times.value(i) as u64;
            let end_ns = end_times.value(i) as u64;
            let name = names.value(i)?;
            let filename = filenames.value(i)?;
            let target = targets.value(i)?;
            let line = lines.value(i);

            if begin_ns < end_ns {
                // Emit async span begin and end events with single writer
                writer
                    .emit_async_span_begin(begin_ns, name, target, filename, line)
                    .await?;
                writer
                    .emit_async_span_end(end_ns, name, target, filename, line)
                    .await?;

                span_count += 1;
                // Flush every 10 async spans to create multiple chunks
                if span_count % 10 == 0 {
                    writer.flush().await?;
                }
            } else {
                warn!("Skipping async span with invalid duration");
            }
        }
    }

    Ok(())
}

/// TableProvider wrapper for PerfettoTraceExecutionPlan
#[derive(Debug)]
pub struct PerfettoTraceTableProvider {
    execution_plan: Arc<PerfettoTraceExecutionPlan>,
}

impl PerfettoTraceTableProvider {
    pub fn new(execution_plan: Arc<PerfettoTraceExecutionPlan>) -> Self {
        Self { execution_plan }
    }
}

#[async_trait::async_trait]
impl TableProvider for PerfettoTraceTableProvider {
    fn as_any(&self) -> &dyn Any {
        self
    }

    fn schema(&self) -> SchemaRef {
        self.execution_plan.schema()
    }

    fn table_type(&self) -> TableType {
        TableType::Base
    }

    async fn scan(
        &self,
        _state: &dyn Session,
        _projection: Option<&Vec<usize>>,
        _filters: &[Expr],
        limit: Option<usize>,
    ) -> DFResult<Arc<dyn ExecutionPlan>> {
        // Wrap the execution plan in a GlobalLimitExec if a limit is provided.
        // DataFusion trusts us to apply the limit - if we ignore it, too many rows
        // will be returned to the client.
        let plan: Arc<dyn ExecutionPlan> = self.execution_plan.clone();
        if let Some(fetch) = limit {
            Ok(Arc::new(GlobalLimitExec::new(plan, 0, Some(fetch))))
        } else {
            Ok(plan)
        }
    }
}