Skip to content

Commit 9a4055e

Browse files
committed
Add example for using a separate threadpool for CPU bound work
1 parent ecc04d4 commit 9a4055e

File tree

4 files changed

+1088
-1
lines changed

4 files changed

+1088
-1
lines changed
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,167 @@
1+
// Licensed to the Apache Software Foundation (ASF) under one
2+
// or more contributor license agreements. See the NOTICE file
3+
// distributed with this work for additional information
4+
// regarding copyright ownership. The ASF licenses this file
5+
// to you under the Apache License, Version 2.0 (the
6+
// "License"); you may not use this file except in compliance
7+
// with the License. You may obtain a copy of the License at
8+
//
9+
// http://www.apache.org/licenses/LICENSE-2.0
10+
//
11+
// Unless required by applicable law or agreed to in writing,
12+
// software distributed under the License is distributed on an
13+
// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
14+
// KIND, either express or implied. See the License for the
15+
// specific language governing permissions and limitations
16+
// under the License.
17+
18+
use arrow::util::pretty::pretty_format_batches;
19+
use datafusion::error::Result;
20+
use datafusion::prelude::*;
21+
use futures::stream::StreamExt;
22+
use datafusion::execution::SendableRecordBatchStream;
23+
//! This example shows how to use a separate thread pool (tokio [`Runtime`])) to
24+
//! run the CPU intensive parts of DataFusion plans.
25+
//!
26+
//! Running DataFusion plans that perform I/O, such as reading parquet files
27+
//! directly from remote object storage (e.g. AWS S3) without care will result
28+
//! in running CPU intensive jobs on the same thread pool, which can lead to the
29+
//! issues described in the [Architecture section] such as throttled bandwidth
30+
//! due to congestion control and increased latencies for processing network
31+
//! messages.
32+
33+
34+
/// Normally, you don't need to worry about the details of the tokio runtime,
35+
/// but for this example it is important to understand how the [`Runtime`]s work.
36+
///
37+
/// There is a "current" runtime that is installed in a thread local variable
38+
/// that is used by the `tokio::spawn` function.
39+
///
40+
/// The `#[tokio::main]` macro actually creates a [`Runtime`] and installs it as
41+
/// as the "current" runtime (on which any `async` futures, streams and tasks
42+
/// are run).
43+
#[tokio::main]
44+
async fn main() -> Result<()> {
45+
let ctx = SessionContext::new()
46+
// enabling URL table means we can select directly from
47+
// paths in SQL queries.
48+
.enable_url_table();
49+
let sql = format!(
50+
"SELECT * FROM '{}/alltypes_plain.parquet'",
51+
datafusion::test_util::parquet_test_data()
52+
);
53+
54+
// Run the same query on the same runtime. Note that calling `await` here
55+
// will effectively run the future (in this case the `async` function) on
56+
// the current runtime.
57+
same_runtime(&ctx, &sql).await?;
58+
59+
// Run the same query on a different runtime. Note that we are still calling
60+
// `await` here, so the the `async` function still runs on the current runtime.
61+
// We use the `DedicatedExecutor` to run the query on a different runtime.
62+
different_runtime(ctx, sql).await?;
63+
Ok(())
64+
}
65+
66+
/// Run queries directly on the current tokio `Runtime`
67+
///
68+
/// This is now most examples in DataFusion are written and works well for
69+
/// development and local query processing.
70+
async fn same_runtime(ctx: &SessionContext, sql: &str) -> Result<()> {
71+
// Calling .sql is an async function as it may also do network
72+
// I/O, for example to contact a remote catalog or do an object store LIST
73+
let df = ctx.sql(sql).await?;
74+
75+
// While many examples call `collect` or `show()`, those methods buffers the
76+
// results. internally DataFusion generates output a RecordBatch at a time
77+
78+
// Calling `execute_stream` on a DataFrame returns a
79+
// `SendableRecordBatchStream`. Depending on the plan, this may also do
80+
// network I/O, for example to begin reading a parquet file from a remote
81+
// object store as well. It is also possible that this function call spawns
82+
// tasks that begin doing CPU intensive work as well
83+
let mut stream: SendableRecordBatchStream = df.execute_stream().await?;
84+
85+
// Calling `next()` drives the plan, producing new `RecordBatch`es using the
86+
// current runtime (and typically also the current thread).
87+
//
88+
// Perhaps somewhat non obvious, calling the `next()` function often will
89+
// result in other tasks being spawned on the current runtime (e.g. for
90+
// `RepartitionExec` to read data from each of its input partitions in
91+
// parallel).
92+
//
93+
// Executing the plan like this results in all CPU intensive work
94+
// running on same (default) Runtime.
95+
while let Some(batch) = stream.next().await {
96+
println!("{}", pretty_format_batches(&[batch?]).unwrap());
97+
}
98+
Ok(())
99+
}
100+
101+
/// Demonstrates how to run queries on a **different** runtime than the current one
102+
///
103+
/// This is typically how you should run DataFusion queries from a network
104+
/// server or when processing data from a remote object store.
105+
async fn different_runtime(ctx: SessionContext, sql: String) -> Result<()> {
106+
// First, we need a new runtime, which we can create with the tokio builder
107+
// however, since we are already in the context of another runtime
108+
// (installed by #[tokio::main]) we create a new thread for the runtime
109+
tokio::task::spawn_blocking(move || {
110+
std::thread::spawn(move || thread_entry(ctx, sql.to_string()))
111+
.join()
112+
.expect("thread did not panic")
113+
})
114+
.await
115+
.expect("task did not panic")
116+
}
117+
118+
/// This is the entry point of thread that we started our second runtime on
119+
fn thread_entry(ctx: SessionContext, sql: String) -> Result<()> {
120+
let runtime = tokio::runtime::Builder::new_multi_thread()
121+
// only enable the time driver (not the I/O driver), meaning this
122+
// runtime will not be able to perform network I/O
123+
.enable_time()
124+
.build()?;
125+
126+
// Now we can run the actual code we want on a different runtime
127+
runtime.block_on(async move { different_runtime_inner(ctx, sql).await })
128+
}
129+
130+
/// this is run on a new, separate runtime
131+
async fn different_runtime_inner(ctx: SessionContext, sql: String) -> Result<()> {
132+
// Setup execution as before
133+
let df = ctx.sql(&sql).await?;
134+
135+
let mut stream = df.execute_stream().await?;
136+
137+
//XXX Note at this point, calling next() will be run on our new threadpool. However, this will also spawn the catalog and object store requests on the same threadpool as well!
138+
139+
// While this will mean we don't interfere with handling of other network requests, it will mean tht the network requests that happen as part of query processing will still be running on the same threadpool
140+
141+
// TODO show this working
142+
143+
// To avoid this, all IO access, both catalog and data (e.g. object_store) must be spawned on to their own runtime, like this:
144+
// TODO....
145+
146+
//
147+
// care is required to avoid calling `next()` (aka polling) from the default IO thread (even if planning / execution is run on that other thread)
148+
// Best practice is to do all of DataFusion planning / execution on a separate pool. Note that some care is required for remote catalogs such as iceberg that
149+
// themselves do network IO
150+
// TODO figure out how to cause an erorr due to io thread
151+
152+
while let Some(batch) = stream.next().await {
153+
println!("{}", pretty_format_batches(&[batch?]).unwrap());
154+
}
155+
156+
//
157+
// You can use the DataFusion "DedicatedExecutor" in this case to handle most of this automatically for you.
158+
159+
// Note that special care must be taken when running Datafusion plans that do async io to transfer the work to their own thread pools.
160+
161+
// Using separate Runtime will avoid other requests being messed up but it won't really help requests made from DataSources such as
162+
// reading parquet files from object_store.
163+
//
164+
// Thus this runtime also disables IO so that we are sure there is no IO work being done on it.
165+
166+
Ok(())
167+
}

datafusion/common-runtime/Cargo.toml

+3-1
Original file line numberDiff line numberDiff line change
@@ -38,6 +38,8 @@ path = "src/lib.rs"
3838
[dependencies]
3939
log = { workspace = true }
4040
tokio = { workspace = true }
41+
parking_lot = { workspace = true }
42+
futures = { workspace = true }
4143

4244
[dev-dependencies]
43-
tokio = { version = "1.36", features = ["rt", "rt-multi-thread", "time"] }
45+
tokio = { version = "1.36", features = ["rt", "rt-multi-thread", "time", "net"] }

0 commit comments

Comments
 (0)