datafusion_common_runtime/
common.rs

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
18use std::{
19    future::Future,
20    pin::Pin,
21    task::{Context, Poll},
22};
23
24use tokio::task::{JoinError, JoinHandle};
25
26use crate::trace_utils::{trace_block, trace_future};
27
28/// Helper that  provides a simple API to spawn a single task and join it.
29/// Provides guarantees of aborting on `Drop` to keep it cancel-safe.
30/// Note that if the task was spawned with `spawn_blocking`, it will only be
31/// aborted if it hasn't started yet.
32///
33/// Technically, it's just a wrapper of a `JoinHandle` overriding drop.
34#[derive(Debug)]
35pub struct SpawnedTask<R> {
36    inner: JoinHandle<R>,
37}
38
39impl<R: 'static> SpawnedTask<R> {
40    pub fn spawn<T>(task: T) -> Self
41    where
42        T: Future<Output = R>,
43        T: Send + 'static,
44        R: Send,
45    {
46        // Ok to use spawn here as SpawnedTask handles aborting/cancelling the task on Drop
47        #[allow(clippy::disallowed_methods)]
48        let inner = tokio::task::spawn(trace_future(task));
49        Self { inner }
50    }
51
52    pub fn spawn_blocking<T>(task: T) -> Self
53    where
54        T: FnOnce() -> R,
55        T: Send + 'static,
56        R: Send,
57    {
58        // Ok to use spawn_blocking here as SpawnedTask handles aborting/cancelling the task on Drop
59        #[allow(clippy::disallowed_methods)]
60        let inner = tokio::task::spawn_blocking(trace_block(task));
61        Self { inner }
62    }
63
64    /// Joins the task, returning the result of join (`Result<R, JoinError>`).
65    /// Same as awaiting the spawned task, but left for backwards compatibility.
66    pub async fn join(self) -> Result<R, JoinError> {
67        self.await
68    }
69
70    /// Joins the task and unwinds the panic if it happens.
71    pub async fn join_unwind(mut self) -> Result<R, JoinError> {
72        self.join_unwind_mut().await
73    }
74
75    /// Joins the task using a mutable reference and unwinds the panic if it happens.
76    ///
77    /// This method is similar to [`join_unwind`](Self::join_unwind), but takes a mutable
78    /// reference instead of consuming `self`. This allows the `SpawnedTask` to remain
79    /// usable after the call.
80    ///
81    /// If called multiple times on the same task:
82    /// - If the task is still running, it will continue waiting for completion
83    /// - If the task has already completed successfully, subsequent calls will
84    ///   continue to return the same `JoinError` indicating the task is finished
85    /// - If the task panicked, the first call will resume the panic, and the
86    ///   program will not reach subsequent calls
87    pub async fn join_unwind_mut(&mut self) -> Result<R, JoinError> {
88        self.await.map_err(|e| {
89            // `JoinError` can be caused either by panic or cancellation. We have to handle panics:
90            if e.is_panic() {
91                std::panic::resume_unwind(e.into_panic());
92            } else {
93                log::warn!("SpawnedTask was polled during shutdown");
94                e
95            }
96        })
97    }
98}
99
100impl<R> Future for SpawnedTask<R> {
101    type Output = Result<R, JoinError>;
102
103    fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
104        Pin::new(&mut self.inner).poll(cx)
105    }
106}
107
108impl<R> Drop for SpawnedTask<R> {
109    fn drop(&mut self) {
110        self.inner.abort();
111    }
112}
113
114#[cfg(test)]
115mod tests {
116    use super::*;
117
118    use std::future::{pending, Pending};
119
120    use tokio::{runtime::Runtime, sync::oneshot};
121
122    #[tokio::test]
123    async fn runtime_shutdown() {
124        let rt = Runtime::new().unwrap();
125        #[allow(clippy::async_yields_async)]
126        let task = rt
127            .spawn(async {
128                SpawnedTask::spawn(async {
129                    let fut: Pending<()> = pending();
130                    fut.await;
131                    unreachable!("should never return");
132                })
133            })
134            .await
135            .unwrap();
136
137        // caller shutdown their DF runtime (e.g. timeout, error in caller, etc)
138        rt.shutdown_background();
139
140        // race condition
141        // poll occurs during shutdown (buffered stream poll calls, etc)
142        assert!(matches!(
143            task.join_unwind().await,
144            Err(e) if e.is_cancelled()
145        ));
146    }
147
148    #[tokio::test]
149    #[should_panic(expected = "foo")]
150    async fn panic_resume() {
151        // this should panic w/o an `unwrap`
152        SpawnedTask::spawn(async { panic!("foo") })
153            .join_unwind()
154            .await
155            .ok();
156    }
157
158    #[tokio::test]
159    async fn cancel_not_started_task() {
160        let (sender, receiver) = oneshot::channel::<i32>();
161        let task = SpawnedTask::spawn(async {
162            // Shouldn't be reached.
163            sender.send(42).unwrap();
164        });
165
166        drop(task);
167
168        // If the task was cancelled, the sender was also dropped,
169        // and awaiting the receiver should result in an error.
170        assert!(receiver.await.is_err());
171    }
172
173    #[tokio::test]
174    async fn cancel_ongoing_task() {
175        let (sender, mut receiver) = tokio::sync::mpsc::channel(1);
176        let task = SpawnedTask::spawn(async move {
177            sender.send(1).await.unwrap();
178            // This line will never be reached because the channel has a buffer
179            // of 1.
180            sender.send(2).await.unwrap();
181        });
182        // Let the task start.
183        assert_eq!(receiver.recv().await.unwrap(), 1);
184        drop(task);
185
186        // The sender was dropped so we receive `None`.
187        assert!(receiver.recv().await.is_none());
188    }
189}