Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

feat: instrument spawned tasks with current tracing span when tracing feature is enabled #14547

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
81 changes: 81 additions & 0 deletions Cargo.lock

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

1 change: 1 addition & 0 deletions README.md
Original file line number Diff line number Diff line change
Expand Up @@ -129,6 +129,7 @@ Optional features:
- `backtrace`: include backtrace information in error messages
- `pyarrow`: conversions between PyArrow and DataFusion types
- `serde`: enable arrow-schema's `serde` feature
- `tracing`: propagates the current span across thread boundaries
Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Question: Is the new tracing feature required? Or should we just make it the default?


[apache avro]: https://avro.apache.org/
[apache parquet]: https://parquet.apache.org/
Expand Down
4 changes: 3 additions & 1 deletion datafusion-examples/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -61,7 +61,7 @@ async-trait = { workspace = true }
bytes = { workspace = true }
dashmap = { workspace = true }
# note only use main datafusion crate for examples
datafusion = { workspace = true, default-features = true, features = ["avro"] }
datafusion = { workspace = true, default-features = true, features = ["avro", "tracing"] }
datafusion-proto = { workspace = true }
env_logger = { workspace = true }
futures = { workspace = true }
Expand All @@ -73,6 +73,8 @@ tempfile = { workspace = true }
test-utils = { path = "../test-utils" }
tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] }
tonic = "0.12.1"
tracing = { version = "0.1" }
tracing-subscriber = { version = "0.3" }
url = { workspace = true }
uuid = "1.7"

Expand Down
127 changes: 127 additions & 0 deletions datafusion-examples/examples/tracing.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,127 @@
// 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 example demonstrates the trace feature in DataFusion’s runtime.
//! When the `trace` feature is enabled, spawned tasks in DataFusion (such as those
//! created during repartitioning or when reading Parquet files) are instrumented
//! with the current tracing span, allowing to propagate any existing tracing context.
//!
//! In this example we create a session configured to use multiple partitions,
//! register a Parquet table (based on the `alltypes_tiny_pages_plain.parquet` file),
//! and run a query that should trigger parallel execution on multiple threads.
//! We wrap the entire query execution within a custom span and log messages.
//! By inspecting the tracing output, we should see that the tasks spawned
//! internally inherit the span context.

use arrow::util::pretty::pretty_format_batches;
use datafusion::arrow::record_batch::RecordBatch;
use datafusion::datasource::file_format::parquet::ParquetFormat;
use datafusion::datasource::listing::ListingOptions;
use datafusion::error::Result;
use datafusion::prelude::*;
use datafusion::test_util::parquet_test_data;
use std::sync::Arc;
use tracing::{info, Level, instrument};

#[tokio::main]
async fn main() -> Result<()> {
// Initialize a tracing subscriber that prints to stdout.
tracing_subscriber::fmt()
.with_thread_ids(true)
.with_thread_names(true)
.with_max_level(Level::DEBUG)
.init();

log::info!("Starting example, this log is not captured by tracing");

// execute the query within a tracing span
let result = run_instrumented_query().await;

info!(
"Finished example. Check the logs above for tracing span details showing \
that tasks were spawned within the 'run_instrumented_query' span on different threads."
);

result
}

#[instrument(level = "info")]
async fn run_instrumented_query() -> Result<()> {
info!("Starting query execution within the custom tracing span");

// The default session will set the number of partitions to `std::thread::available_parallelism()`.
let ctx = SessionContext::new();

// Get the path to the test parquet data.
let test_data = parquet_test_data();
// Build listing options that pick up only the "alltypes_tiny_pages_plain.parquet" file.
let file_format = ParquetFormat::default().with_enable_pruning(true);
let listing_options = ListingOptions::new(Arc::new(file_format))
.with_file_extension("alltypes_tiny_pages_plain.parquet");

info!("Registering Parquet table 'alltypes' from {test_data} in {listing_options:?}");

// Register a listing table using an absolute URL.
let table_path = format!("file://{test_data}/");
ctx.register_listing_table(
"alltypes",
&table_path,
listing_options.clone(),
None,
None,
)
.await
.expect("register_listing_table failed");

info!("Registered Parquet table 'alltypes' from {table_path}");

// Run a query that will trigger parallel execution on multiple threads.
let sql = "SELECT COUNT(*), bool_col, date_string_col, string_col
FROM (
SELECT bool_col, date_string_col, string_col FROM alltypes
UNION ALL
SELECT bool_col, date_string_col, string_col FROM alltypes
) AS t
GROUP BY bool_col, date_string_col, string_col
ORDER BY 1,2,3,4 DESC
LIMIT 5;";
info!(%sql, "Executing SQL query");
let df = ctx.sql(sql).await?;

let results: Vec<RecordBatch> = df.collect().await?;
info!("Query execution complete");

// Print out the results and tracing output.
datafusion::common::assert_batches_eq!(
[
"+----------+----------+-----------------+------------+",
"| count(*) | bool_col | date_string_col | string_col |",
"+----------+----------+-----------------+------------+",
"| 2 | false | 01/01/09 | 9 |",
"| 2 | false | 01/01/09 | 7 |",
"| 2 | false | 01/01/09 | 5 |",
"| 2 | false | 01/01/09 | 3 |",
"| 2 | false | 01/01/09 | 1 |",
"+----------+----------+-----------------+------------+",
],
&results
);

info!("Query results:\n{}", pretty_format_batches(&results)?);

Ok(())
}
5 changes: 5 additions & 0 deletions datafusion/common-runtime/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -31,13 +31,18 @@ rust-version = { workspace = true }
[lints]
workspace = true

[features]
tracing = ["dep:tracing", "dep:tracing-futures"]

[lib]
name = "datafusion_common_runtime"
path = "src/lib.rs"

[dependencies]
log = { workspace = true }
tokio = { workspace = true }
tracing = { version = "0.1", optional = true }
tracing-futures = { version = "0.2", optional = true }

[dev-dependencies]
tokio = { version = "1.36", features = ["rt", "rt-multi-thread", "time"] }
4 changes: 3 additions & 1 deletion datafusion/common-runtime/src/common.rs
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,8 @@

use std::future::Future;

use tokio::task::{JoinError, JoinSet};
use tokio::task::{JoinError};
use crate::JoinSet;

/// Helper that provides a simple API to spawn a single task and join it.
/// Provides guarantees of aborting on `Drop` to keep it cancel-safe.
Expand All @@ -36,6 +37,7 @@ impl<R: 'static> SpawnedTask<R> {
R: Send,
{
let mut inner = JoinSet::new();

inner.spawn(task);
Self { inner }
}
Expand Down
Loading