|
| 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