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(self) -> Result<R, JoinError> {
72 self.await.map_err(|e| {
73 // `JoinError` can be caused either by panic or cancellation. We have to handle panics:
74 if e.is_panic() {
75 std::panic::resume_unwind(e.into_panic());
76 } else {
77 // Cancellation may be caused by two reasons:
78 // 1. Abort is called, but since we consumed `self`, it's not our case (`JoinHandle` not accessible outside).
79 // 2. The runtime is shutting down.
80 log::warn!("SpawnedTask was polled during shutdown");
81 e
82 }
83 })
84 }
85}
86
87impl<R> Future for SpawnedTask<R> {
88 type Output = Result<R, JoinError>;
89
90 fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
91 Pin::new(&mut self.inner).poll(cx)
92 }
93}
94
95impl<R> Drop for SpawnedTask<R> {
96 fn drop(&mut self) {
97 self.inner.abort();
98 }
99}
100
101#[cfg(test)]
102mod tests {
103 use super::*;
104
105 use std::future::{pending, Pending};
106
107 use tokio::{runtime::Runtime, sync::oneshot};
108
109 #[tokio::test]
110 async fn runtime_shutdown() {
111 let rt = Runtime::new().unwrap();
112 #[allow(clippy::async_yields_async)]
113 let task = rt
114 .spawn(async {
115 SpawnedTask::spawn(async {
116 let fut: Pending<()> = pending();
117 fut.await;
118 unreachable!("should never return");
119 })
120 })
121 .await
122 .unwrap();
123
124 // caller shutdown their DF runtime (e.g. timeout, error in caller, etc)
125 rt.shutdown_background();
126
127 // race condition
128 // poll occurs during shutdown (buffered stream poll calls, etc)
129 assert!(matches!(
130 task.join_unwind().await,
131 Err(e) if e.is_cancelled()
132 ));
133 }
134
135 #[tokio::test]
136 #[should_panic(expected = "foo")]
137 async fn panic_resume() {
138 // this should panic w/o an `unwrap`
139 SpawnedTask::spawn(async { panic!("foo") })
140 .join_unwind()
141 .await
142 .ok();
143 }
144
145 #[tokio::test]
146 async fn cancel_not_started_task() {
147 let (sender, receiver) = oneshot::channel::<i32>();
148 let task = SpawnedTask::spawn(async {
149 // Shouldn't be reached.
150 sender.send(42).unwrap();
151 });
152
153 drop(task);
154
155 // If the task was cancelled, the sender was also dropped,
156 // and awaiting the receiver should result in an error.
157 assert!(receiver.await.is_err());
158 }
159
160 #[tokio::test]
161 async fn cancel_ongoing_task() {
162 let (sender, mut receiver) = tokio::sync::mpsc::channel(1);
163 let task = SpawnedTask::spawn(async move {
164 sender.send(1).await.unwrap();
165 // This line will never be reached because the channel has a buffer
166 // of 1.
167 sender.send(2).await.unwrap();
168 });
169 // Let the task start.
170 assert_eq!(receiver.recv().await.unwrap(), 1);
171 drop(task);
172
173 // The sender was dropped so we receive `None`.
174 assert!(receiver.recv().await.is_none());
175 }
176}