datafusion-tracing 53.0.0

DataFusion tracing of execution plans
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
// Licensed to the Apache Software Foundation (ASF) under one
// or more contributor license agreements.  See the NOTICE file
// distributed with this work for additional information
// regarding copyright ownership.  The ASF licenses this file
// to you under the Apache License, Version 2.0 (the
// "License"); you may not use this file except in compliance
// with the License.  You may obtain a copy of the License at
//
//   http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing,
// software distributed under the License is distributed on an
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
// KIND, either express or implied.  See the License for the
// specific language governing permissions and limitations
// under the License.
//
// This product includes software developed at Datadog (https://www.datadoghq.com/) Copyright 2025 Datadog, Inc.

use crate::{
    metrics::{MetricsRecorder, MetricsRecordingStream},
    node::{NodeRecorder, NodeRecordingStream},
    options::InstrumentationOptions,
    preview::{PreviewFn, PreviewRecorder, PreviewRecordingStream},
    utils::is_internal_optimizer_check,
};
use datafusion::{
    arrow::datatypes::SchemaRef,
    common::Statistics,
    config::ConfigOptions,
    error::Result,
    execution::{SendableRecordBatchStream, TaskContext},
    physical_expr::{Distribution, OrderingRequirements, PhysicalSortExpr},
    physical_plan::{
        DisplayAs, DisplayFormatType, ExecutionPlan, ExecutionPlanProperties,
        PhysicalExpr, PlanProperties,
        execution_plan::{CardinalityEffect, InvariantLevel},
        filter_pushdown::{
            ChildPushdownResult, FilterDescription, FilterPushdownPhase,
            FilterPushdownPropagation,
        },
        metrics::MetricsSet,
        projection::ProjectionExec,
        sort_pushdown::SortOrderPushdownResult,
        stream::RecordBatchStreamAdapter,
    },
};
use delegate::delegate;
use std::{
    any::Any,
    collections::HashMap,
    fmt::{self, Debug},
    sync::{Arc, OnceLock},
};
use tracing::{Span, field};
use tracing_futures::Instrument;

/// Type alias for a function that creates a tracing span.
pub(crate) type SpanCreateFn = dyn Fn() -> Span + Send + Sync;

/// An [`ExecutionPlan`] wrapper that instruments execution with tracing spans and metrics recording.
pub struct InstrumentedExec {
    /// The inner execution plan to delegate execution to.
    inner: Arc<dyn ExecutionPlan>,

    /// Tracing span lazily initialized during execution, shared safely across concurrent partition executions.
    span: OnceLock<Span>,

    record_metrics: bool,

    /// Metrics recorder lazily initialized during execution, shared safely across concurrent partition executions.
    metrics_recorder: OnceLock<Arc<MetricsRecorder>>,

    /// Node recorder lazily initialized during execution, shared safely across concurrent partition executions.
    node_recorder: OnceLock<Arc<NodeRecorder>>,

    preview_limit: usize,
    preview_fn: Option<Arc<PreviewFn>>,

    /// Preview recorder lazily initialized during execution, shared safely across concurrent partition executions.
    preview_recorder: OnceLock<Arc<PreviewRecorder>>,

    /// Function to create and initialize tracing spans.
    span_create_fn: Arc<SpanCreateFn>,
}

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

impl InstrumentedExec {
    /// Creates a new `InstrumentedExec` that wraps an execution plan with tracing and metrics.
    pub fn new(
        inner: Arc<dyn ExecutionPlan>,
        span_create_fn: Arc<SpanCreateFn>,
        options: &InstrumentationOptions,
    ) -> InstrumentedExec {
        Self {
            inner,
            span: OnceLock::new(),
            record_metrics: options.record_metrics,
            metrics_recorder: OnceLock::new(),
            node_recorder: OnceLock::new(),
            preview_limit: options.preview_limit,
            preview_fn: options.preview_fn.clone(),
            preview_recorder: OnceLock::new(),
            span_create_fn,
        }
    }

    /// Creates a new `InstrumentedExec` with the same configuration as this instance but with a different inner execution plan.
    ///
    /// This method is used when the optimizer needs to replace the inner execution plan while preserving
    /// all the instrumentation settings (metrics recording, preview limits, span creation function, etc.).
    fn with_new_inner(&self, inner: Arc<dyn ExecutionPlan>) -> Arc<dyn ExecutionPlan> {
        Arc::new(InstrumentedExec::new(
            inner,
            self.span_create_fn.clone(),
            &InstrumentationOptions {
                record_metrics: self.record_metrics,
                preview_limit: self.preview_limit,
                preview_fn: self.preview_fn.clone(),
                custom_fields: HashMap::new(), // custom fields are not used by `InstrumentedExec`, only by the higher-level `instrument_with_spans` macro family
            },
        ))
    }

    /// Retrieves the tracing span, initializing it if necessary.
    fn get_span(&self) -> Span {
        self.span
            .get_or_init(|| self.create_populated_span())
            .clone()
    }

    /// Wraps the given stream with metrics recording if metrics are available.
    /// The input span is shared across all partitions,
    /// and metrics will be aggregated across all partitions before being reported.
    fn metrics_recording_stream(
        &self,
        inner_stream: SendableRecordBatchStream,
        span: &Span,
    ) -> SendableRecordBatchStream {
        if !self.record_metrics {
            return inner_stream;
        }
        let recorder = self
            .metrics_recorder
            .get_or_init(|| {
                Arc::new(MetricsRecorder::new(self.inner.clone(), span.clone()))
            })
            .clone();
        Box::pin(MetricsRecordingStream::new(inner_stream, recorder))
    }

    /// Wraps the given stream with batch preview recording.
    /// The input span is shared across all partitions,
    /// and the preview limit will be applied globally on all partitions before the preview is reported.
    fn preview_recording_stream(
        &self,
        inner_stream: SendableRecordBatchStream,
        span: &Span,
        partition: usize,
    ) -> SendableRecordBatchStream {
        if self.preview_limit == 0 {
            return inner_stream;
        }
        let recorder = self
            .preview_recorder
            .get_or_init(|| {
                let partition_count = self.inner.output_partitioning().partition_count();
                Arc::new(
                    PreviewRecorder::builder(span.clone(), partition_count)
                        .limit(self.preview_limit)
                        .preview_fn(self.preview_fn.clone())
                        .build(),
                )
            })
            .clone();
        Box::pin(PreviewRecordingStream::new(
            inner_stream,
            recorder,
            partition,
        ))
    }

    /// Wraps the given stream with a completion recorder so fields that are only
    /// fully qualified after execution (such as `datafusion.node`) are recorded
    /// once all partitions have finished executing.
    fn node_recording_stream(
        &self,
        inner_stream: SendableRecordBatchStream,
        span: &Span,
    ) -> SendableRecordBatchStream {
        let recorder = self
            .node_recorder
            .get_or_init(|| Arc::new(NodeRecorder::new(self.inner.clone(), span.clone())))
            .clone();
        Box::pin(NodeRecordingStream::new(inner_stream, recorder))
    }

    /// Creates a tracing span populated with metadata about the execution plan.
    fn create_populated_span(&self) -> Span {
        let span = self.span_create_fn.as_ref()();

        span.record("otel.name", field::display(self.inner.name()));
        span.record(
            "datafusion.partitioning",
            field::display(self.inner.properties().partitioning.clone()),
        );
        span.record(
            "datafusion.emission_type",
            field::debug(self.inner.properties().emission_type),
        );
        span.record(
            "datafusion.boundedness",
            field::debug(self.inner.properties().boundedness),
        );

        span
    }

    /// Returns true if the plan is an `InstrumentedExec` wrapper.
    ///
    /// This relies on the internal optimization context being active in the current thread.
    pub(crate) fn is_instrumented(plan: &dyn ExecutionPlan) -> bool {
        plan.as_any().is::<InstrumentedExec>()
    }
}

impl ExecutionPlan for InstrumentedExec {
    // Most ExecutionPlan methods are delegated to the inner plan. Methods that must return a
    // wrapped plan or provide custom behavior are implemented manually below.
    delegate! {
        to self.inner {
            fn schema(&self) -> SchemaRef;
            fn properties(&self) -> &Arc<PlanProperties>;
            fn name(&self) -> &str;
            fn check_invariants(&self, check: InvariantLevel) -> Result<()>;
            fn required_input_distribution(&self) -> Vec<Distribution>;
            fn required_input_ordering(&self) -> Vec<Option<OrderingRequirements>>;
            fn maintains_input_order(&self) -> Vec<bool>;
            fn benefits_from_input_partitioning(&self) -> Vec<bool>;
            fn children(&self) -> Vec<&Arc<dyn ExecutionPlan>>;
            fn metrics(&self) -> Option<MetricsSet>;
            fn partition_statistics(&self, partition: Option<usize>) -> Result<Statistics>;
            fn supports_limit_pushdown(&self) -> bool;
            fn fetch(&self) -> Option<usize>;
            fn cardinality_effect(&self) -> CardinalityEffect;
            fn gather_filters_for_pushdown(
                &self,
                phase: FilterPushdownPhase,
                parent_filters: Vec<Arc<dyn PhysicalExpr>>,
                config: &ConfigOptions,
            ) -> Result<FilterDescription>;
        }
    }

    fn static_name() -> &'static str {
        "InstrumentedExec"
    }

    /// Delegate to the inner plan for repartitioning and rewrap with an InstrumentedExec.
    fn repartitioned(
        &self,
        target_partitions: usize,
        config: &ConfigOptions,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        if let Some(new_inner) = self
            .inner
            .clone()
            .repartitioned(target_partitions, config)?
        {
            Ok(Some(self.with_new_inner(new_inner)))
        } else {
            Ok(None)
        }
    }

    /// Delegate to the inner plan for fetching and rewrap with an InstrumentedExec.
    fn with_fetch(&self, limit: Option<usize>) -> Option<Arc<dyn ExecutionPlan>> {
        if let Some(new_inner) = self.inner.clone().with_fetch(limit) {
            Some(self.with_new_inner(new_inner))
        } else {
            None
        }
    }

    /// Delegate to the inner plan for swapping with a projection and rewrap with an InstrumentedExec.
    fn try_swapping_with_projection(
        &self,
        projection: &ProjectionExec,
    ) -> Result<Option<Arc<dyn ExecutionPlan>>> {
        if let Some(new_inner) = self
            .inner
            .clone()
            .try_swapping_with_projection(projection)?
        {
            Ok(Some(self.with_new_inner(new_inner)))
        } else {
            Ok(None)
        }
    }

    /// Delegate to the inner plan for handling child pushdown result and rewrap with an InstrumentedExec.
    fn handle_child_pushdown_result(
        &self,
        phase: FilterPushdownPhase,
        child_pushdown_result: ChildPushdownResult,
        config: &ConfigOptions,
    ) -> Result<FilterPushdownPropagation<Arc<dyn ExecutionPlan>>> {
        // If the inner plan updated itself, rewrap the updated node to preserve instrumentation
        let FilterPushdownPropagation {
            filters,
            updated_node,
        } = self.inner.handle_child_pushdown_result(
            phase,
            child_pushdown_result,
            config,
        )?;
        let updated_node = updated_node.map(|n| self.with_new_inner(n));
        Ok(FilterPushdownPropagation {
            filters,
            updated_node,
        })
    }

    /// Delegate to the inner plan for sort pushdown and rewrap with an InstrumentedExec.
    fn try_pushdown_sort(
        &self,
        order: &[PhysicalSortExpr],
    ) -> Result<SortOrderPushdownResult<Arc<dyn ExecutionPlan>>> {
        let result = self.inner.try_pushdown_sort(order)?;
        Ok(match result {
            SortOrderPushdownResult::Exact { inner } => SortOrderPushdownResult::Exact {
                inner: self.with_new_inner(inner),
            },
            SortOrderPushdownResult::Inexact { inner } => {
                SortOrderPushdownResult::Inexact {
                    inner: self.with_new_inner(inner),
                }
            }
            SortOrderPushdownResult::Unsupported => SortOrderPushdownResult::Unsupported,
        })
    }

    /// Delegate to the inner plan for creating new children and rewrap with an InstrumentedExec.
    fn with_new_children(
        self: Arc<Self>,
        children: Vec<Arc<dyn ExecutionPlan>>,
    ) -> Result<Arc<dyn ExecutionPlan>> {
        let new_inner = self.inner.clone().with_new_children(children)?;
        Ok(self.with_new_inner(new_inner))
    }

    /// Delegate to the inner plan for resetting state and rewrap with an InstrumentedExec.
    fn reset_state(self: Arc<Self>) -> Result<Arc<dyn ExecutionPlan>> {
        let new_inner = self.inner.clone().reset_state()?;
        Ok(self.with_new_inner(new_inner))
    }

    /// Delegate to the inner plan for injecting run-time state and rewrap with an InstrumentedExec.
    fn with_new_state(
        &self,
        state: Arc<dyn Any + Send + Sync>,
    ) -> Option<Arc<dyn ExecutionPlan>> {
        let new_inner = self.inner.with_new_state(state)?;
        Some(self.with_new_inner(new_inner))
    }

    /// Returns the plan as any to allow for downcasting.
    ///
    /// During optimization passes, this returns `self` (the `InstrumentedExec`) to
    /// allow the optimizer to identify already-instrumented nodes.
    ///
    /// Otherwise, this delegates to the inner plan to provide "transparent downcasting",
    /// allowing users to downcast an instrumented node to its original type.
    fn as_any(&self) -> &dyn Any {
        if is_internal_optimizer_check() {
            self
        } else {
            self.inner.as_any()
        }
    }

    /// Executes the plan for a given partition and context, instrumented with tracing and metrics recording.
    fn execute(
        &self,
        partition: usize,
        context: Arc<TaskContext>,
    ) -> Result<SendableRecordBatchStream> {
        let span = self.get_span();

        let inner_stream = span.in_scope(|| self.inner.execute(partition, context))?;

        // Wrap the stream with node recording so `datafusion.node` is recorded only after
        // completion, once it is fully qualified.
        let node_stream = self.node_recording_stream(inner_stream, &span);

        // Wrap the stream with metrics recording capability (only if inner metrics are available).
        let metrics_stream = self.metrics_recording_stream(node_stream, &span);

        // Wrap the stream with batch preview recording (only if preview limit is set).
        let preview_stream =
            self.preview_recording_stream(metrics_stream, &span, partition);

        Ok(Box::pin(RecordBatchStreamAdapter::new(
            self.inner.schema(),
            preview_stream.instrument(span),
        )))
    }
}

impl DisplayAs for InstrumentedExec {
    delegate! {
        to self.inner {
            fn fmt_as(&self, format: DisplayFormatType, f: &mut fmt::Formatter<'_>) -> fmt::Result;
        }
    }
}