Skip to content
Closed
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
185 changes: 185 additions & 0 deletions datafusion-examples/examples/thread_pools.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,185 @@
// 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 shows how to use a separate thread pool (tokio [`Runtime`])) to
//! run the CPU intensive parts of DataFusion plans.
//!
//! Running DataFusion plans that perform I/O, such as reading parquet files
//! directly from remote object storage (e.g. AWS S3) without care will result
//! in running CPU intensive jobs on the same thread pool, which can lead to the
//! issues described in the [Architecture section] such as throttled bandwidth
//! due to congestion control and increased latencies for processing network
//! messages.
use arrow::util::pretty::pretty_format_batches;
use datafusion::error::Result;
use datafusion::execution::dedicated_executor::DedicatedExecutor;
use datafusion::execution::runtime_env::RuntimeEnvBuilder;
use datafusion::execution::{SendableRecordBatchStream, SessionStateBuilder};
use datafusion::prelude::*;
use futures::stream::StreamExt;
use object_store::http::HttpBuilder;
use object_store::ObjectStore;
use std::sync::Arc;
use url::Url;

/// Normally, you don't need to worry about the details of the tokio runtime,
/// but for this example it is important to understand how the [`Runtime`]s work.
///
/// There is a "current" runtime that is installed in a thread local variable
/// that is used by the `tokio::spawn` function.
///
/// The `#[tokio::main]` macro actually creates a [`Runtime`] and installs it as
/// as the "current" runtime (on which any `async` futures, streams and tasks
/// are run).
#[tokio::main]
async fn main() -> Result<()> {
// The first two examples only do local file IO. Enable the URL table so we
// can select directly from filenames in SQL.
let sql = format!(
"SELECT * FROM '{}/alltypes_plain.parquet'",
datafusion::test_util::parquet_test_data()
);

// Run the same query on the same runtime. Note that calling `await` here
// will effectively run the future (in this case the `async` function) on
// the current runtime
same_runtime(&sql).await?;

// Run the same query on a different runtime.
// Run the same query on a different runtime including remote IO
different_runtime_advanced().await?;

Ok(())
}

/// Run queries directly on the current tokio `Runtime`
///
/// This is now most examples in DataFusion are written and works well for
/// development and local query processing.
async fn same_runtime(sql: &str) -> Result<()> {
let ctx = SessionContext::new().enable_url_table();

// Calling .sql is an async function as it may also do network
// I/O, for example to contact a remote catalog or do an object store LIST
let df = ctx.sql(sql).await?;

// While many examples call `collect` or `show()`, those methods buffers the
// results. internally DataFusion generates output a RecordBatch at a time

// Calling `execute_stream` on a DataFrame returns a
// `SendableRecordBatchStream`. Depending on the plan, this may also do
// network I/O, for example to begin reading a parquet file from a remote
// object store as well. It is also possible that this function call spawns
// tasks that begin doing CPU intensive work as well
let mut stream: SendableRecordBatchStream = df.execute_stream().await?;

// Calling `next()` drives the plan, producing new `RecordBatch`es using the
// current runtime (and typically also the current thread).
//
// Perhaps somewhat non obvious, calling the `next()` function often will
// result in other tasks being spawned on the current runtime (e.g. for
// `RepartitionExec` to read data from each of its input partitions in
// parallel).
//
// Executing the plan like this results in all CPU intensive work
// running on same (default) Runtime.
while let Some(batch) = stream.next().await {
println!("{}", pretty_format_batches(&[batch?]).unwrap());
}
Ok(())
}

/// Demonstrates how to run queries on a **different** runtime than the current one
///
async fn different_runtime_advanced() -> Result<()> {
// In this example, we will configure access to a remote object store
// over the network during the plan

let dedicated_executor = DedicatedExecutor::builder().build();

// setup http object store
let base_url = Url::parse("https://github.com").unwrap();
let http_store: Arc<dyn ObjectStore> =
Arc::new(HttpBuilder::new().with_url(base_url.clone()).build()?);

// By default, the object store will use the "current runtime" for IO operations
// if we use a dedicated executor to run the plan, the eventual object store requests will also use the
// dedicated executor's runtime
//
// To avoid this, we can wrap the object store to run on the "IO" runtime
//
// (if we don't do this the example fails with an error like
//
// ctx.register_object_store(&base_url, http_store);
// A Tokio 1.x context was found, but timers are disabled. Call `enable_time` on the runtime builder to enable timers.

//let http_store = dedicated_executor.wrap_object_store(http_store);

// we must also register the dedicated executor with the runtime
let runtime_env = RuntimeEnvBuilder::new()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

In this example the DedicatedExecutor is registered directly with the RuntimeEnv

I actually think this is a much nicer API than having to wrap various calls with dedicated executors

.with_dedicated_executor(dedicated_executor.clone())
.build_arc()?;

// Tell datafusion about processing http:// urls with this wrapped object store
runtime_env.register_object_store(&base_url, http_store);

let ctx = SessionContext::from(
SessionStateBuilder::new()
.with_runtime_env(runtime_env)
.with_default_features()
.build(),
)
.enable_url_table();

// Plan (and execute) the query on the dedicated runtime
// TODO it would be great to figure out how to run this as part of `ctx.sql`
// Plan / execute the query
let url = "https://github.com/apache/arrow-testing/raw/master/data/csv/aggregate_test_100.csv";
let df = ctx
.sql(&format!("SELECT c1,c2,c3 FROM '{url}' LIMIT 5"))
.await?;
let stream: SendableRecordBatchStream = df.execute_stream().await?;

// We have now planned the query on the dedicated runtime, Yay! but we still need to
// drive the stream (aka call `next()` to get the results).

// However, as mentioned above, calling `next()` resolves the Stream (and
// any work it may do) on a thread in the current (default) runtime.
//
// To drive the stream on the dedicated runtime, we need to wrap it using a
// `DedicatedExecutor::wrap_stream` stream function
//
// Note if you don't do this you will likely see a panic about `No IO runtime registered.`
// because the threads in the current (main) tokio runtime have not had the IO runtime
// installed
let mut stream = dedicated_executor.run_sendable_record_batch_stream(stream);

// Note you can run other streams on the DedicatedExecutor as well using the
// DedicatedExecutor:YYYXXX function. This is helpful for example, if you
// need to do non trivial CPU work on the results of the stream (e.g.
// calling a FlightDataEncoder to convert the results to flight to send it
// over the network),

while let Some(batch) = stream.next().await {
println!("{}", pretty_format_batches(&[batch?]).unwrap());
}

Ok(())
}

// TODO add an example of a how to call IO / CPU bound work directly using DedicatedExecutor
// (e.g. to create a listing table directly)
55 changes: 32 additions & 23 deletions datafusion/core/src/datasource/dynamic_file.rs
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ use std::sync::Arc;

use async_trait::async_trait;
use datafusion_catalog::{SessionStore, UrlTableFactory};
use datafusion_common::plan_datafusion_err;
use datafusion_common::{internal_err, plan_datafusion_err};

use crate::datasource::listing::{ListingTable, ListingTableConfig, ListingTableUrl};
use crate::datasource::TableProvider;
Expand Down Expand Up @@ -55,33 +55,42 @@ impl UrlTableFactory for DynamicListTableFactory {
return Ok(None);
};

let state = &self
let session = self
.session_store()
.get_session()
.upgrade()
.and_then(|session| {
session
.read()
.as_any()
.downcast_ref::<SessionState>()
.cloned()
})
.ok_or_else(|| plan_datafusion_err!("get current SessionStore error"))?;

match ListingTableConfig::new(table_url.clone())
.infer_options(state)
let runtime_env = Arc::clone(&session.read().runtime_env());

let Some(state) = session
.read()
.as_any()
.downcast_ref::<SessionState>()
.cloned()
else {
return internal_err!("Expected SessionState, got something else");
};

// Do remove catalog operations on a different runtime
runtime_env
Copy link
Contributor Author

@alamb alamb Dec 8, 2024

Choose a reason for hiding this comment

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

This shows how annotating the I/O call sites in DataFusion would look like (calling spawn_io internally)

This call could be converted easily because it doesn't pass in &self or capture any fields

.spawn_io(async move {
match ListingTableConfig::new(table_url.clone())
.infer_options(&state)
.await
{
Ok(cfg) => {
let cfg = cfg
.infer_partitions_from_path(&state)
.await?
.infer_schema(&state)
.await?;
ListingTable::try_new(cfg)
.map(|table| Some(Arc::new(table) as Arc<dyn TableProvider>))
}
Err(_) => Ok(None),
}
})
.await
{
Ok(cfg) => {
let cfg = cfg
.infer_partitions_from_path(state)
.await?
.infer_schema(state)
.await?;
ListingTable::try_new(cfg)
.map(|table| Some(Arc::new(table) as Arc<dyn TableProvider>))
}
Err(_) => Ok(None),
}
}
}
20 changes: 16 additions & 4 deletions datafusion/core/src/datasource/listing/table.rs
Original file line number Diff line number Diff line change
Expand Up @@ -679,7 +679,7 @@ impl ListingOptions {
/// # Ok(())
/// # }
/// ```
#[derive(Debug)]
#[derive(Debug, Clone)]
pub struct ListingTable {
table_paths: Vec<ListingTableUrl>,
/// File fields only
Expand Down Expand Up @@ -843,10 +843,22 @@ impl TableProvider for ListingTable {
});
// TODO (https://github.com/apache/datafusion/issues/11600) remove downcast_ref from here?
let session_state = state.as_any().downcast_ref::<SessionState>().unwrap();
let (mut partitioned_file_lists, statistics) = self
.list_files_for_scan(session_state, &partition_filters, limit)
// TODO avoid these clones when possible.
let session_state_captured = session_state.clone();
Copy link
Contributor Author

Choose a reason for hiding this comment

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

There are quite a few more call sites like this that would need to be annotated for the correct runtime. However I didn't bother to do it unless we could figure out how to make the rest of the API reasonable (aka run_cpu ,etc)

let partition_filters_captured = partition_filters.clone();
let self_captured = self.clone();
let (mut partitioned_file_lists, statistics) = state
.runtime_env()
.spawn_io(async move {
self_captured
.list_files_for_scan(
&session_state_captured,
&partition_filters_captured,
limit,
)
.await
})
.await?;

// if no files need to be read, return an `EmptyExec`
if partitioned_file_lists.is_empty() {
let projected_schema = project_schema(&self.schema(), projection)?;
Expand Down
10 changes: 7 additions & 3 deletions datafusion/core/src/execution/context/mod.rs
Original file line number Diff line number Diff line change
Expand Up @@ -608,10 +608,14 @@ impl SessionContext {
sql: &str,
options: SQLOptions,
) -> Result<DataFrame> {
let plan = self.state().create_logical_plan(sql).await?;
options.verify_plan(&plan)?;
self.runtime_env()
Copy link
Contributor Author

Choose a reason for hiding this comment

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

And here is an example of how I would like to be able to call high level DataFusion APIs to to run cpu bound work, but I can't get the lifetimes to work out

.spawn_cpu2(async {
let plan = self.state().create_logical_plan(sql).await?;
options.verify_plan(&plan)?;

self.execute_logical_plan(plan).await
self.execute_logical_plan(plan).await
})
.await
}

/// Creates logical expressions from SQL query text.
Expand Down
7 changes: 7 additions & 0 deletions datafusion/execution/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ path = "src/lib.rs"

[dependencies]
arrow = { workspace = true }
async-trait = {workspace = true}
chrono = { workspace = true }
dashmap = { workspace = true }
datafusion-common = { workspace = true, default-features = true }
Expand All @@ -45,6 +46,12 @@ futures = { workspace = true }
log = { workspace = true }
object_store = { workspace = true }
parking_lot = { workspace = true }
pin-project-lite = "^0.2.7"
rand = { workspace = true }
tempfile = { workspace = true }

url = { workspace = true }
tokio = { workspace = true }
# todo figure out if we need to use tokio_stream / could use record batch receiver stream
tokio-stream = {version = "0.1"}

Loading
Loading