Skip to content

Commit b5d4ae1

Browse files
committed
Example for using a separate threadpool for CPU bound work (try 2)
1 parent 1dad545 commit b5d4ae1

File tree

4 files changed

+2041
-0
lines changed

4 files changed

+2041
-0
lines changed

datafusion-examples/Cargo.toml

+1
Original file line numberDiff line numberDiff line change
@@ -77,6 +77,7 @@ prost = { workspace = true }
7777
tempfile = { workspace = true }
7878
test-utils = { path = "../test-utils" }
7979
tokio = { workspace = true, features = ["rt-multi-thread", "parking_lot"] }
80+
tokio-stream = { version = "0.1" }
8081
tonic = "0.12.1"
8182
url = { workspace = true }
8283
uuid = "1.7"
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,238 @@
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+
//! This example shows how to use separate thread pools (tokio [`Runtime`]))s to
19+
//! run the IO and CPU intensive parts of DataFusion plans.
20+
//!
21+
//! # Background
22+
//!
23+
//! DataFusion, by default, plans and executes all operations (both CPU and IO)
24+
//! on the same thread pool. This makes it fast and easy to get started, but
25+
//! can cause issues when running at scale, especially when fetching and operating
26+
//! on data directly from remote sources.
27+
//!
28+
//! Specifically, DataFusion plans that perform I/O, such as reading parquet files
29+
//! directly from remote object storage (e.g. AWS S3) on the same threadpool
30+
//! as CPU intensive work, can lead to the issues described in the
31+
//! [Architecture section] such as throttled network bandwidth (due to congestion
32+
//! control) and increased latencies or timeouts while processing network
33+
//! messages.
34+
//!
35+
//! It is possible, but more complex, as shows in this example, to separate
36+
//! the IO and CPU bound work on separate runtimes to avoid these issues.
37+
use crate::thread_pools_lib::dedicated_executor::DedicatedExecutor;
38+
use arrow::util::pretty::pretty_format_batches;
39+
use datafusion::error::Result;
40+
use datafusion::execution::SendableRecordBatchStream;
41+
use datafusion::prelude::*;
42+
use futures::stream::StreamExt;
43+
use object_store::http::HttpBuilder;
44+
use object_store::ObjectStore;
45+
use std::sync::Arc;
46+
use url::Url;
47+
48+
mod thread_pools_lib;
49+
50+
/// Normally, you don't need to worry about the details of the tokio runtime,
51+
/// but for this example it is important to understand how the [`Runtime`]s work.
52+
///
53+
/// There is a "current" runtime that is installed in a thread local variable
54+
/// that is used by the `tokio::spawn` function.
55+
///
56+
/// The `#[tokio::main]` macro creates a [`Runtime`] and installs it as
57+
/// as the "current" runtime in a thread local variable, on which any `async`
58+
/// [`Future`], [`Stream]`s and [`Task]`s are run.
59+
#[tokio::main]
60+
async fn main() -> Result<()> {
61+
// The first two examples read local files, so enable the URL table feature
62+
// so we can treat filenames as tables in SQL.
63+
let ctx = SessionContext::new().enable_url_table();
64+
let sql = format!(
65+
"SELECT * FROM '{}/alltypes_plain.parquet'",
66+
datafusion::test_util::parquet_test_data()
67+
);
68+
69+
// Run a query on the current runtime. Calling `await` means the future
70+
// (in this case the `async` function and all spawned work in DataFusion
71+
// plans) on the current runtime.
72+
same_runtime(&ctx, &sql).await?;
73+
74+
// Run the same query but this time on a different runtime. Since we call
75+
// `await` here, the `async` function itself runs on the current runtime,
76+
// but internally `different_runtime_basic` uses a `DedicatedExecutor` to
77+
// run the execute the DataFusion plan on a different Runtime.
78+
different_runtime_basic(ctx, sql).await?;
79+
80+
// Run the same query on a different runtime, including remote IO.
81+
//
82+
// This is best practice for production systems
83+
different_runtime_advanced().await?;
84+
85+
Ok(())
86+
}
87+
88+
/// Run queries directly on the current tokio `Runtime`
89+
///
90+
/// This is how most examples in DataFusion are written and works well for
91+
/// development and local query processing.
92+
async fn same_runtime(ctx: &SessionContext, sql: &str) -> Result<()> {
93+
// Calling .sql is an async function as it may also do network
94+
// I/O, for example to contact a remote catalog or do an object store LIST
95+
let df = ctx.sql(sql).await?;
96+
97+
// While many examples call `collect` or `show()`, those methods buffers the
98+
// results. Internally DataFusion generates output a RecordBatch at a time
99+
100+
// Calling `execute_stream` return a `SendableRecordBatchStream`. Depending
101+
// on the plan, this may also do network I/O, for example to begin reading a
102+
// parquet file from a remote object store as well. It is also possible that
103+
// this function call spawns tasks as well.
104+
let mut stream: SendableRecordBatchStream = df.execute_stream().await?;
105+
106+
// Calling `next()` drives the plan, producing new `RecordBatch`es using the
107+
// current runtime (and typically also the current thread).
108+
//
109+
// Perhaps somewhat non obviously, calling the `next()` function can also
110+
// result in other tasks being spawned on the current runtime (e.g. for
111+
// `RepartitionExec` to read data from each of its input partitions in
112+
// parallel).
113+
//
114+
// Executing the plan like this results in all CPU intensive work
115+
// running on same Runtime, in this case whichever one ran the work
116+
while let Some(batch) = stream.next().await {
117+
println!("{}", pretty_format_batches(&[batch?]).unwrap());
118+
}
119+
Ok(())
120+
}
121+
122+
/// Run queries on a **different** runtime than the current one
123+
///
124+
/// This is an intermediate example for explanatory purposes. Production systems
125+
/// should follow the recommendations on [`different_runtime_advanced`] when
126+
/// running DataFusion queries from a network server or when processing data
127+
/// from a remote object store.
128+
async fn different_runtime_basic(ctx: SessionContext, sql: String) -> Result<()> {
129+
// Since we are already in the context of runtime (installed by
130+
// #[tokio::main]), First, we need a new runtime, which is managed by
131+
// a DedicatedExecutor (a library in this example)
132+
let dedicated_executor = DedicatedExecutor::builder().build();
133+
134+
// Now, we run the query on the new runtime
135+
dedicated_executor
136+
.spawn(async move {
137+
// this closure runs on the different thread pool
138+
let df = ctx.sql(&sql).await?;
139+
let mut stream: SendableRecordBatchStream = df.execute_stream().await?;
140+
141+
// Calling `next()` to drive the plan in this closure drives the
142+
// execution on the other thread pool
143+
//
144+
// NOTE any IO run by this plan (for example, reading from an
145+
// `ObjectStore`) will be done on this new thread pool as well.
146+
while let Some(batch) = stream.next().await {
147+
println!("{}", pretty_format_batches(&[batch?]).unwrap());
148+
}
149+
Ok(()) as Result<()>
150+
})
151+
// even though we are `await`ing here on the "current" pool, internally
152+
// the DedicatedExecutor runs the work on the separate thread pool and
153+
// the `await` simply notifies it when the work is done
154+
.await??;
155+
156+
// When done with a DedicatedExecutor, it should be shut down cleanly to give
157+
// any outstanding tasks a chance to complete.
158+
dedicated_executor.join().await;
159+
160+
Ok(())
161+
}
162+
163+
/// Demonstrates running queries so that
164+
/// 1. IO operations happen on the current thread pool
165+
/// 2. CPU bound tasks happen on a different thread pool
166+
async fn different_runtime_advanced() -> Result<()> {
167+
// In this example, we will query a file via https, reading
168+
// the data directly from the plan
169+
170+
let ctx = SessionContext::new().enable_url_table();
171+
172+
// setup http object store
173+
let base_url = Url::parse("https://github.com").unwrap();
174+
let http_store: Arc<dyn ObjectStore> =
175+
Arc::new(HttpBuilder::new().with_url(base_url.clone()).build()?);
176+
177+
let dedicated_executor = DedicatedExecutor::builder().build();
178+
179+
// By default, the object store will use the runtime that calls `await` for
180+
// IO operations. As shown above, using a DedicatedExecutor will run the
181+
// plan (and all its IO on the same runtime).
182+
//
183+
// To avoid this, we can wrap the object store to run on the "IO" runtime
184+
//
185+
// You can use the DedicatedExecutor::spawn_io method to run other IO
186+
// operations.
187+
//
188+
// Note if we don't do this the example fails with an error like
189+
//
190+
// ctx.register_object_store(&base_url, http_store);
191+
// A Tokio 1.x context was found, but timers are disabled. Call `enable_time` on the runtime builder to enable timers.
192+
let http_store = dedicated_executor.wrap_object_store_for_io(http_store);
193+
194+
// Tell DataDusion to process `http://` urls with this wrapped object store
195+
ctx.register_object_store(&base_url, http_store);
196+
197+
// Plan and begin to execute the query on the dedicated runtime
198+
let stream = dedicated_executor
199+
.spawn(async move {
200+
// Plan / execute the query
201+
let url = "https://github.com/apache/arrow-testing/raw/master/data/csv/aggregate_test_100.csv";
202+
let df = ctx
203+
.sql(&format!("SELECT c1,c2,c3 FROM '{url}' LIMIT 5"))
204+
.await?;
205+
206+
// since we wrapped the object store, and I/O will actually happen
207+
// on the current runtime.
208+
let stream: SendableRecordBatchStream = df.execute_stream().await?;
209+
210+
Ok(stream) as Result<_>
211+
}).await??;
212+
213+
// We have now planned the query on the dedicated executor Yay! However,
214+
// most applications will still drive the stream (aka call `next()` to get
215+
// the results) from the current runtime, for example to send results back
216+
// over arrow-flight.
217+
218+
// However, as mentioned above, calling `next()` drives the Stream (and any
219+
// work it may do) on a thread in the current (default) runtime.
220+
//
221+
// To drive the Stream on the dedicated runtime, we need to wrap it using
222+
// the `DedicatedExecutor::wrap_stream` stream function
223+
//
224+
// Note if you don't do this you will likely see a panic about `No IO runtime registered.`
225+
// because the threads in the current (main) tokio runtime have not had the IO runtime
226+
// installed
227+
let mut stream = dedicated_executor.run_cpu_sendable_record_batch_stream(stream);
228+
229+
// Note you can run other streams on the DedicatedExecutor as well using the
230+
// same function. This is helpful for example, if you need to do non trivial
231+
// CPU work on the results of the stream (e.g. calling a FlightDataEncoder
232+
// to convert the results to flight to send it over the network),
233+
while let Some(batch) = stream.next().await {
234+
println!("{}", pretty_format_batches(&[batch?]).unwrap());
235+
}
236+
237+
Ok(())
238+
}

0 commit comments

Comments
 (0)