feat: support `async fn` in macros with coroutine implementation
This commit is contained in:
parent
abe518d164
commit
627841f1e2
|
@ -31,6 +31,9 @@ unindent = { version = "0.2.1", optional = true }
|
||||||
# support crate for multiple-pymethods feature
|
# support crate for multiple-pymethods feature
|
||||||
inventory = { version = "0.3.0", optional = true }
|
inventory = { version = "0.3.0", optional = true }
|
||||||
|
|
||||||
|
# coroutine implementation
|
||||||
|
futures-util = "0.3"
|
||||||
|
|
||||||
# crate integrations that can be added using the eponymous features
|
# crate integrations that can be added using the eponymous features
|
||||||
anyhow = { version = "1.0", optional = true }
|
anyhow = { version = "1.0", optional = true }
|
||||||
chrono = { version = "0.4.25", default-features = false, optional = true }
|
chrono = { version = "0.4.25", default-features = false, optional = true }
|
||||||
|
@ -54,6 +57,7 @@ serde = { version = "1.0", features = ["derive"] }
|
||||||
serde_json = "1.0.61"
|
serde_json = "1.0.61"
|
||||||
rayon = "1.6.1"
|
rayon = "1.6.1"
|
||||||
widestring = "0.5.1"
|
widestring = "0.5.1"
|
||||||
|
futures = "0.3.28"
|
||||||
|
|
||||||
[build-dependencies]
|
[build-dependencies]
|
||||||
pyo3-build-config = { path = "pyo3-build-config", version = "0.21.0-dev", features = ["resolve-config"] }
|
pyo3-build-config = { path = "pyo3-build-config", version = "0.21.0-dev", features = ["resolve-config"] }
|
||||||
|
|
|
@ -19,6 +19,7 @@
|
||||||
- [Conversion traits](conversions/traits.md)]
|
- [Conversion traits](conversions/traits.md)]
|
||||||
- [Python exceptions](exception.md)
|
- [Python exceptions](exception.md)
|
||||||
- [Calling Python from Rust](python_from_rust.md)
|
- [Calling Python from Rust](python_from_rust.md)
|
||||||
|
- [Using `async` and `await`](async-await.md)
|
||||||
- [GIL, mutability and object types](types.md)
|
- [GIL, mutability and object types](types.md)
|
||||||
- [Parallelism](parallelism.md)
|
- [Parallelism](parallelism.md)
|
||||||
- [Debugging](debugging.md)
|
- [Debugging](debugging.md)
|
||||||
|
|
|
@ -0,0 +1,78 @@
|
||||||
|
# Using `async` and `await`
|
||||||
|
|
||||||
|
*This feature is still in active development. See [the related issue](https://github.com/PyO3/pyo3/issues/1632).*
|
||||||
|
|
||||||
|
`#[pyfunction]` and `#[pymethods]` attributes also support `async fn`.
|
||||||
|
|
||||||
|
```rust
|
||||||
|
# #![allow(dead_code)]
|
||||||
|
use std::{thread, time::Duration};
|
||||||
|
use futures::channel::oneshot;
|
||||||
|
use pyo3::prelude::*;
|
||||||
|
|
||||||
|
#[pyfunction]
|
||||||
|
async fn sleep(seconds: f64, result: Option<PyObject>) -> Option<PyObject> {
|
||||||
|
let (tx, rx) = oneshot::channel();
|
||||||
|
thread::spawn(move || {
|
||||||
|
thread::sleep(Duration::from_secs_f64(seconds));
|
||||||
|
tx.send(()).unwrap();
|
||||||
|
});
|
||||||
|
rx.await.unwrap();
|
||||||
|
result
|
||||||
|
}
|
||||||
|
```
|
||||||
|
|
||||||
|
*Python awaitables instantiated with this method can only be awaited in *asyncio* context. Other Python async runtime may be supported in the future.*
|
||||||
|
|
||||||
|
## `Send + 'static` constraint
|
||||||
|
|
||||||
|
Resulting future of an `async fn` decorated by `#[pyfunction]` must be `Send + 'static` to be embedded in a Python object.
|
||||||
|
|
||||||
|
As a consequence, `async fn` parameters and return types must also be `Send + 'static`, so it is not possible to have a signature like `async fn does_not_compile(arg: &PyAny, py: Python<'_>) -> &PyAny`.
|
||||||
|
|
||||||
|
It also means that methods cannot use `&self`/`&mut self`, *but this restriction should be dropped in the future.*
|
||||||
|
|
||||||
|
|
||||||
|
## Implicit GIL holding
|
||||||
|
|
||||||
|
Even if it is not possible to pass a `py: Python<'_>` parameter to `async fn`, the GIL is still held during the execution of the future – it's also the case for regular `fn` without `Python<'_>`/`&PyAny` parameter, yet the GIL is held.
|
||||||
|
|
||||||
|
It is still possible to get a `Python` marker using [`Python::with_gil`]({{#PYO3_DOCS_URL}}/pyo3/struct.Python.html#method.with_gil); because `with_gil` is reentrant and optimized, the cost will be negligible.
|
||||||
|
|
||||||
|
## Release the GIL across `.await`
|
||||||
|
|
||||||
|
There is currently no simple way to release the GIL when awaiting a future, *but solutions are currently in development*.
|
||||||
|
|
||||||
|
Here is the advised workaround for now:
|
||||||
|
|
||||||
|
```rust,ignore
|
||||||
|
use std::{future::Future, pin::{Pin, pin}, task::{Context, Poll}};
|
||||||
|
use pyo3::prelude::*;
|
||||||
|
|
||||||
|
struct AllowThreads<F>(F);
|
||||||
|
|
||||||
|
impl<F> Future for AllowThreads<F>
|
||||||
|
where
|
||||||
|
F: Future + Unpin + Send,
|
||||||
|
F::Output: Send,
|
||||||
|
{
|
||||||
|
type Output = F::Output;
|
||||||
|
|
||||||
|
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||||
|
let waker = cx.waker();
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
gil.allow_threads(|| pin!(&mut self.0).poll(&mut Context::from_waker(waker)))
|
||||||
|
})
|
||||||
|
}
|
||||||
|
}
|
||||||
|
```
|
||||||
|
|
||||||
|
## Cancellation
|
||||||
|
|
||||||
|
*To be implemented*
|
||||||
|
|
||||||
|
## The `Coroutine` type
|
||||||
|
|
||||||
|
To make a Rust future awaitable in Python, PyO3 defines a [`Coroutine`]({{#PYO3_DOCS_URL}}/pyo3/coroutine/struct.Coroutine.html) type, which implements the Python [coroutine protocol](https://docs.python.org/3/library/collections.abc.html#collections.abc.Coroutine). Each `coroutine.send` call is translated to `Future::poll` call, while `coroutine.throw` call reraise the exception *(this behavior will be configurable with cancellation support)*.
|
||||||
|
|
||||||
|
*The type does not yet have a public constructor until the design is finalized.*
|
|
@ -1,5 +1,7 @@
|
||||||
# Using `async` and `await`
|
# Using `async` and `await`
|
||||||
|
|
||||||
|
*`async`/`await` support is currently being integrated in PyO3. See the [dedicated documentation](../async-await.md)*
|
||||||
|
|
||||||
If you are working with a Python library that makes use of async functions or wish to provide
|
If you are working with a Python library that makes use of async functions or wish to provide
|
||||||
Python bindings for an async Rust library, [`pyo3-asyncio`](https://github.com/awestlake87/pyo3-asyncio)
|
Python bindings for an async Rust library, [`pyo3-asyncio`](https://github.com/awestlake87/pyo3-asyncio)
|
||||||
likely has the tools you need. It provides conversions between async functions in both Python and
|
likely has the tools you need. It provides conversions between async functions in both Python and
|
||||||
|
|
|
@ -0,0 +1 @@
|
||||||
|
Support `async fn` in macros with coroutine implementation
|
|
@ -228,6 +228,7 @@ pub struct FnSpec<'a> {
|
||||||
pub output: syn::Type,
|
pub output: syn::Type,
|
||||||
pub convention: CallingConvention,
|
pub convention: CallingConvention,
|
||||||
pub text_signature: Option<TextSignatureAttribute>,
|
pub text_signature: Option<TextSignatureAttribute>,
|
||||||
|
pub asyncness: Option<syn::Token![async]>,
|
||||||
pub unsafety: Option<syn::Token![unsafe]>,
|
pub unsafety: Option<syn::Token![unsafe]>,
|
||||||
pub deprecations: Deprecations,
|
pub deprecations: Deprecations,
|
||||||
}
|
}
|
||||||
|
@ -317,6 +318,7 @@ impl<'a> FnSpec<'a> {
|
||||||
signature,
|
signature,
|
||||||
output: ty,
|
output: ty,
|
||||||
text_signature,
|
text_signature,
|
||||||
|
asyncness: sig.asyncness,
|
||||||
unsafety: sig.unsafety,
|
unsafety: sig.unsafety,
|
||||||
deprecations,
|
deprecations,
|
||||||
})
|
})
|
||||||
|
@ -445,7 +447,11 @@ impl<'a> FnSpec<'a> {
|
||||||
let func_name = &self.name;
|
let func_name = &self.name;
|
||||||
|
|
||||||
let rust_call = |args: Vec<TokenStream>| {
|
let rust_call = |args: Vec<TokenStream>| {
|
||||||
quotes::map_result_into_ptr(quotes::ok_wrap(quote! { function(#self_arg #(#args),*) }))
|
let mut call = quote! { function(#self_arg #(#args),*) };
|
||||||
|
if self.asyncness.is_some() {
|
||||||
|
call = quote! { _pyo3::impl_::coroutine::wrap_future(#call) };
|
||||||
|
}
|
||||||
|
quotes::map_result_into_ptr(quotes::ok_wrap(call))
|
||||||
};
|
};
|
||||||
|
|
||||||
let rust_name = if let Some(cls) = cls {
|
let rust_name = if let Some(cls) = cls {
|
||||||
|
|
|
@ -6,7 +6,7 @@ use crate::{
|
||||||
deprecations::Deprecations,
|
deprecations::Deprecations,
|
||||||
method::{self, CallingConvention, FnArg},
|
method::{self, CallingConvention, FnArg},
|
||||||
pymethod::check_generic,
|
pymethod::check_generic,
|
||||||
utils::{ensure_not_async_fn, get_pyo3_crate},
|
utils::get_pyo3_crate,
|
||||||
};
|
};
|
||||||
use proc_macro2::TokenStream;
|
use proc_macro2::TokenStream;
|
||||||
use quote::{format_ident, quote};
|
use quote::{format_ident, quote};
|
||||||
|
@ -179,8 +179,6 @@ pub fn impl_wrap_pyfunction(
|
||||||
options: PyFunctionOptions,
|
options: PyFunctionOptions,
|
||||||
) -> syn::Result<TokenStream> {
|
) -> syn::Result<TokenStream> {
|
||||||
check_generic(&func.sig)?;
|
check_generic(&func.sig)?;
|
||||||
ensure_not_async_fn(&func.sig)?;
|
|
||||||
|
|
||||||
let PyFunctionOptions {
|
let PyFunctionOptions {
|
||||||
pass_module,
|
pass_module,
|
||||||
name,
|
name,
|
||||||
|
@ -230,6 +228,7 @@ pub fn impl_wrap_pyfunction(
|
||||||
signature,
|
signature,
|
||||||
output: ty,
|
output: ty,
|
||||||
text_signature,
|
text_signature,
|
||||||
|
asyncness: func.sig.asyncness,
|
||||||
unsafety: func.sig.unsafety,
|
unsafety: func.sig.unsafety,
|
||||||
deprecations: Deprecations::new(),
|
deprecations: Deprecations::new(),
|
||||||
};
|
};
|
||||||
|
|
|
@ -2,7 +2,7 @@ use std::borrow::Cow;
|
||||||
|
|
||||||
use crate::attributes::{NameAttribute, RenamingRule};
|
use crate::attributes::{NameAttribute, RenamingRule};
|
||||||
use crate::method::{CallingConvention, ExtractErrorMode};
|
use crate::method::{CallingConvention, ExtractErrorMode};
|
||||||
use crate::utils::{ensure_not_async_fn, PythonDoc};
|
use crate::utils::PythonDoc;
|
||||||
use crate::{
|
use crate::{
|
||||||
method::{FnArg, FnSpec, FnType, SelfType},
|
method::{FnArg, FnSpec, FnType, SelfType},
|
||||||
pyfunction::PyFunctionOptions,
|
pyfunction::PyFunctionOptions,
|
||||||
|
@ -188,7 +188,6 @@ pub fn gen_py_method(
|
||||||
options: PyFunctionOptions,
|
options: PyFunctionOptions,
|
||||||
) -> Result<GeneratedPyMethod> {
|
) -> Result<GeneratedPyMethod> {
|
||||||
check_generic(sig)?;
|
check_generic(sig)?;
|
||||||
ensure_not_async_fn(sig)?;
|
|
||||||
ensure_function_options_valid(&options)?;
|
ensure_function_options_valid(&options)?;
|
||||||
let method = PyMethod::parse(sig, meth_attrs, options)?;
|
let method = PyMethod::parse(sig, meth_attrs, options)?;
|
||||||
let spec = &method.spec;
|
let spec = &method.spec;
|
||||||
|
|
|
@ -1,6 +1,6 @@
|
||||||
use proc_macro2::{Span, TokenStream};
|
use proc_macro2::{Span, TokenStream};
|
||||||
use quote::ToTokens;
|
use quote::ToTokens;
|
||||||
use syn::{punctuated::Punctuated, spanned::Spanned, Token};
|
use syn::{punctuated::Punctuated, Token};
|
||||||
|
|
||||||
use crate::attributes::{CrateAttribute, RenamingRule};
|
use crate::attributes::{CrateAttribute, RenamingRule};
|
||||||
|
|
||||||
|
@ -137,17 +137,6 @@ impl quote::ToTokens for PythonDoc {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
pub fn ensure_not_async_fn(sig: &syn::Signature) -> syn::Result<()> {
|
|
||||||
if let Some(asyncness) = &sig.asyncness {
|
|
||||||
bail_spanned!(
|
|
||||||
asyncness.span() => "`async fn` is not yet supported for Python functions.\n\n\
|
|
||||||
Additional crates such as `pyo3-asyncio` can be used to integrate async Rust and \
|
|
||||||
Python. For more information, see https://github.com/PyO3/pyo3/issues/1632"
|
|
||||||
);
|
|
||||||
};
|
|
||||||
Ok(())
|
|
||||||
}
|
|
||||||
|
|
||||||
pub fn unwrap_ty_group(mut ty: &syn::Type) -> &syn::Type {
|
pub fn unwrap_ty_group(mut ty: &syn::Type) -> &syn::Type {
|
||||||
while let syn::Type::Group(g) = ty {
|
while let syn::Type::Group(g) = ty {
|
||||||
ty = &*g.elem;
|
ty = &*g.elem;
|
||||||
|
|
|
@ -0,0 +1,137 @@
|
||||||
|
//! Python coroutine implementation, used notably when wrapping `async fn`
|
||||||
|
//! with `#[pyfunction]`/`#[pymethods]`.
|
||||||
|
use std::{
|
||||||
|
any::Any,
|
||||||
|
future::Future,
|
||||||
|
panic,
|
||||||
|
pin::Pin,
|
||||||
|
sync::Arc,
|
||||||
|
task::{Context, Poll},
|
||||||
|
};
|
||||||
|
|
||||||
|
use futures_util::FutureExt;
|
||||||
|
use pyo3_macros::{pyclass, pymethods};
|
||||||
|
|
||||||
|
use crate::{
|
||||||
|
coroutine::waker::AsyncioWaker,
|
||||||
|
exceptions::{PyRuntimeError, PyStopIteration},
|
||||||
|
panic::PanicException,
|
||||||
|
pyclass::IterNextOutput,
|
||||||
|
types::PyIterator,
|
||||||
|
IntoPy, Py, PyAny, PyErr, PyObject, PyResult, Python,
|
||||||
|
};
|
||||||
|
|
||||||
|
mod waker;
|
||||||
|
|
||||||
|
const COROUTINE_REUSED_ERROR: &str = "cannot reuse already awaited coroutine";
|
||||||
|
|
||||||
|
type FutureOutput = Result<PyResult<PyObject>, Box<dyn Any + Send>>;
|
||||||
|
|
||||||
|
/// Python coroutine wrapping a [`Future`].
|
||||||
|
#[pyclass(crate = "crate")]
|
||||||
|
pub struct Coroutine {
|
||||||
|
future: Option<Pin<Box<dyn Future<Output = FutureOutput> + Send>>>,
|
||||||
|
waker: Option<Arc<AsyncioWaker>>,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl Coroutine {
|
||||||
|
/// Wrap a future into a Python coroutine.
|
||||||
|
///
|
||||||
|
/// Coroutine `send` polls the wrapped future, ignoring the value passed
|
||||||
|
/// (should always be `None` anyway).
|
||||||
|
///
|
||||||
|
/// `Coroutine `throw` drop the wrapped future and reraise the exception passed
|
||||||
|
pub(crate) fn from_future<F, T, E>(future: F) -> Self
|
||||||
|
where
|
||||||
|
F: Future<Output = Result<T, E>> + Send + 'static,
|
||||||
|
T: IntoPy<PyObject>,
|
||||||
|
PyErr: From<E>,
|
||||||
|
{
|
||||||
|
let wrap = async move {
|
||||||
|
let obj = future.await?;
|
||||||
|
// SAFETY: GIL is acquired when future is polled (see `Coroutine::poll`)
|
||||||
|
Ok(obj.into_py(unsafe { Python::assume_gil_acquired() }))
|
||||||
|
};
|
||||||
|
Self {
|
||||||
|
future: Some(Box::pin(panic::AssertUnwindSafe(wrap).catch_unwind())),
|
||||||
|
waker: None,
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
fn poll(
|
||||||
|
&mut self,
|
||||||
|
py: Python<'_>,
|
||||||
|
throw: Option<PyObject>,
|
||||||
|
) -> PyResult<IterNextOutput<PyObject, PyObject>> {
|
||||||
|
// raise if the coroutine has already been run to completion
|
||||||
|
let future_rs = match self.future {
|
||||||
|
Some(ref mut fut) => fut,
|
||||||
|
None => return Err(PyRuntimeError::new_err(COROUTINE_REUSED_ERROR)),
|
||||||
|
};
|
||||||
|
// reraise thrown exception it
|
||||||
|
if let Some(exc) = throw {
|
||||||
|
self.close();
|
||||||
|
return Err(PyErr::from_value(exc.as_ref(py)));
|
||||||
|
}
|
||||||
|
// create a new waker, or try to reset it in place
|
||||||
|
if let Some(waker) = self.waker.as_mut().and_then(Arc::get_mut) {
|
||||||
|
waker.reset();
|
||||||
|
} else {
|
||||||
|
self.waker = Some(Arc::new(AsyncioWaker::new()));
|
||||||
|
}
|
||||||
|
let waker = futures_util::task::waker(self.waker.clone().unwrap());
|
||||||
|
// poll the Rust future and forward its results if ready
|
||||||
|
if let Poll::Ready(res) = future_rs.as_mut().poll(&mut Context::from_waker(&waker)) {
|
||||||
|
self.close();
|
||||||
|
return match res {
|
||||||
|
Ok(res) => Ok(IterNextOutput::Return(res?)),
|
||||||
|
Err(err) => Err(PanicException::from_panic_payload(err)),
|
||||||
|
};
|
||||||
|
}
|
||||||
|
// otherwise, initialize the waker `asyncio.Future`
|
||||||
|
if let Some(future) = self.waker.as_ref().unwrap().initialize_future(py)? {
|
||||||
|
// `asyncio.Future` must be awaited; fortunately, it implements `__iter__ = __await__`
|
||||||
|
// and will yield itself if its result has not been set in polling above
|
||||||
|
if let Some(future) = PyIterator::from_object(future).unwrap().next() {
|
||||||
|
// future has not been leaked into Python for now, and Rust code can only call
|
||||||
|
// `set_result(None)` in `ArcWake` implementation, so it's safe to unwrap
|
||||||
|
return Ok(IterNextOutput::Yield(future.unwrap().into()));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
// if waker has been waken during future polling, this is roughly equivalent to
|
||||||
|
// `await asyncio.sleep(0)`, so just yield `None`.
|
||||||
|
Ok(IterNextOutput::Yield(py.None().into()))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(crate) fn iter_result(result: IterNextOutput<PyObject, PyObject>) -> PyResult<PyObject> {
|
||||||
|
match result {
|
||||||
|
IterNextOutput::Yield(ob) => Ok(ob),
|
||||||
|
IterNextOutput::Return(ob) => Err(PyStopIteration::new_err(ob)),
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
#[pymethods(crate = "crate")]
|
||||||
|
impl Coroutine {
|
||||||
|
fn send(&mut self, py: Python<'_>, _value: &PyAny) -> PyResult<PyObject> {
|
||||||
|
iter_result(self.poll(py, None)?)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn throw(&mut self, py: Python<'_>, exc: PyObject) -> PyResult<PyObject> {
|
||||||
|
iter_result(self.poll(py, Some(exc))?)
|
||||||
|
}
|
||||||
|
|
||||||
|
fn close(&mut self) {
|
||||||
|
// the Rust future is dropped, and the field set to `None`
|
||||||
|
// to indicate the coroutine has been run to completion
|
||||||
|
drop(self.future.take());
|
||||||
|
}
|
||||||
|
|
||||||
|
fn __await__(self_: Py<Self>) -> Py<Self> {
|
||||||
|
self_
|
||||||
|
}
|
||||||
|
|
||||||
|
fn __next__(&mut self, py: Python<'_>) -> PyResult<IterNextOutput<PyObject, PyObject>> {
|
||||||
|
self.poll(py, None)
|
||||||
|
}
|
||||||
|
}
|
|
@ -0,0 +1,97 @@
|
||||||
|
use crate::sync::GILOnceCell;
|
||||||
|
use crate::types::PyCFunction;
|
||||||
|
use crate::{intern, wrap_pyfunction, Py, PyAny, PyObject, PyResult, Python};
|
||||||
|
use futures_util::task::ArcWake;
|
||||||
|
use pyo3_macros::pyfunction;
|
||||||
|
use std::sync::Arc;
|
||||||
|
|
||||||
|
/// Lazy `asyncio.Future` wrapper, implementing [`ArcWake`] by calling `Future.set_result`.
|
||||||
|
///
|
||||||
|
/// asyncio future is let uninitialized until [`initialize_future`][1] is called.
|
||||||
|
/// If [`wake`][2] is called before future initialization (during Rust future polling),
|
||||||
|
/// [`initialize_future`][1] will return `None` (it is roughly equivalent to `asyncio.sleep(0)`)
|
||||||
|
///
|
||||||
|
/// [1]: AsyncioWaker::initialize_future
|
||||||
|
/// [2]: AsyncioWaker::wake
|
||||||
|
pub struct AsyncioWaker(GILOnceCell<Option<LoopAndFuture>>);
|
||||||
|
|
||||||
|
impl AsyncioWaker {
|
||||||
|
pub(super) fn new() -> Self {
|
||||||
|
Self(GILOnceCell::new())
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(super) fn reset(&mut self) {
|
||||||
|
self.0.take();
|
||||||
|
}
|
||||||
|
|
||||||
|
pub(super) fn initialize_future<'a>(&'a self, py: Python<'a>) -> PyResult<Option<&'a PyAny>> {
|
||||||
|
let init = || LoopAndFuture::new(py).map(Some);
|
||||||
|
let loop_and_future = self.0.get_or_try_init(py, init)?.as_ref();
|
||||||
|
Ok(loop_and_future.map(|LoopAndFuture { future, .. }| future.as_ref(py)))
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
impl ArcWake for AsyncioWaker {
|
||||||
|
fn wake_by_ref(arc_self: &Arc<Self>) {
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
if let Some(loop_and_future) = arc_self.0.get_or_init(gil, || None) {
|
||||||
|
loop_and_future
|
||||||
|
.set_result(gil)
|
||||||
|
.expect("unexpected error in coroutine waker");
|
||||||
|
}
|
||||||
|
});
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
struct LoopAndFuture {
|
||||||
|
event_loop: PyObject,
|
||||||
|
future: PyObject,
|
||||||
|
}
|
||||||
|
|
||||||
|
impl LoopAndFuture {
|
||||||
|
fn new(py: Python<'_>) -> PyResult<Self> {
|
||||||
|
static GET_RUNNING_LOOP: GILOnceCell<PyObject> = GILOnceCell::new();
|
||||||
|
let import = || -> PyResult<_> {
|
||||||
|
let module = py.import("asyncio")?;
|
||||||
|
Ok(module.getattr("get_running_loop")?.into())
|
||||||
|
};
|
||||||
|
let event_loop = GET_RUNNING_LOOP.get_or_try_init(py, import)?.call0(py)?;
|
||||||
|
let future = event_loop.call_method0(py, "create_future")?;
|
||||||
|
Ok(Self { event_loop, future })
|
||||||
|
}
|
||||||
|
|
||||||
|
fn set_result(&self, py: Python<'_>) -> PyResult<()> {
|
||||||
|
static RELEASE_WAITER: GILOnceCell<Py<PyCFunction>> = GILOnceCell::new();
|
||||||
|
let release_waiter = RELEASE_WAITER
|
||||||
|
.get_or_try_init(py, || wrap_pyfunction!(release_waiter, py).map(Into::into))?;
|
||||||
|
// `Future.set_result` must be called in event loop thread,
|
||||||
|
// so it requires `call_soon_threadsafe`
|
||||||
|
let call_soon_threadsafe = self.event_loop.call_method1(
|
||||||
|
py,
|
||||||
|
intern!(py, "call_soon_threadsafe"),
|
||||||
|
(release_waiter, self.future.as_ref(py)),
|
||||||
|
);
|
||||||
|
if let Err(err) = call_soon_threadsafe {
|
||||||
|
// `call_soon_threadsafe` will raise if the event loop is closed;
|
||||||
|
// instead of catching an unspecific `RuntimeError`, check directly if it's closed.
|
||||||
|
let is_closed = self.event_loop.call_method0(py, "is_closed")?;
|
||||||
|
if !is_closed.extract(py)? {
|
||||||
|
return Err(err);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
Ok(())
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/// Call `future.set_result` if the future is not done.
|
||||||
|
///
|
||||||
|
/// Future can be cancelled by the event loop before being waken.
|
||||||
|
/// See <https://github.com/python/cpython/blob/main/Lib/asyncio/tasks.py#L452C5-L452C5>
|
||||||
|
#[pyfunction(crate = "crate")]
|
||||||
|
fn release_waiter(future: &PyAny) -> PyResult<()> {
|
||||||
|
let done = future.call_method0(intern!(future.py(), "done"))?;
|
||||||
|
if !done.extract::<bool>()? {
|
||||||
|
future.call_method1(intern!(future.py(), "set_result"), (future.py().None(),))?;
|
||||||
|
}
|
||||||
|
Ok(())
|
||||||
|
}
|
|
@ -6,6 +6,8 @@
|
||||||
//! APIs may may change at any time without documentation in the CHANGELOG and without
|
//! APIs may may change at any time without documentation in the CHANGELOG and without
|
||||||
//! breaking semver guarantees.
|
//! breaking semver guarantees.
|
||||||
|
|
||||||
|
#[cfg(feature = "macros")]
|
||||||
|
pub mod coroutine;
|
||||||
pub mod deprecations;
|
pub mod deprecations;
|
||||||
pub mod extract_argument;
|
pub mod extract_argument;
|
||||||
pub mod freelist;
|
pub mod freelist;
|
||||||
|
|
|
@ -0,0 +1,19 @@
|
||||||
|
use crate::coroutine::Coroutine;
|
||||||
|
use crate::impl_::wrap::OkWrap;
|
||||||
|
use crate::{IntoPy, PyErr, PyObject, Python};
|
||||||
|
use std::future::Future;
|
||||||
|
|
||||||
|
/// Used to wrap the result of async `#[pyfunction]` and `#[pymethods]`.
|
||||||
|
pub fn wrap_future<F, R, T>(future: F) -> Coroutine
|
||||||
|
where
|
||||||
|
F: Future<Output = R> + Send + 'static,
|
||||||
|
R: OkWrap<T>,
|
||||||
|
T: IntoPy<PyObject>,
|
||||||
|
PyErr: From<R::Error>,
|
||||||
|
{
|
||||||
|
let future = async move {
|
||||||
|
// SAFETY: GIL is acquired when future is polled (see `Coroutine::poll`)
|
||||||
|
future.await.wrap(unsafe { Python::assume_gil_acquired() })
|
||||||
|
};
|
||||||
|
Coroutine::from_future(future)
|
||||||
|
}
|
|
@ -397,6 +397,8 @@ pub mod buffer;
|
||||||
pub mod callback;
|
pub mod callback;
|
||||||
pub mod conversion;
|
pub mod conversion;
|
||||||
mod conversions;
|
mod conversions;
|
||||||
|
#[cfg(feature = "macros")]
|
||||||
|
pub mod coroutine;
|
||||||
#[macro_use]
|
#[macro_use]
|
||||||
#[doc(hidden)]
|
#[doc(hidden)]
|
||||||
pub mod derive_utils;
|
pub mod derive_utils;
|
||||||
|
@ -469,6 +471,7 @@ pub mod doc_test {
|
||||||
doctests! {
|
doctests! {
|
||||||
"README.md" => readme_md,
|
"README.md" => readme_md,
|
||||||
"guide/src/advanced.md" => guide_advanced_md,
|
"guide/src/advanced.md" => guide_advanced_md,
|
||||||
|
"guide/src/async-await.md" => guide_async_await_md,
|
||||||
"guide/src/building_and_distribution.md" => guide_building_and_distribution_md,
|
"guide/src/building_and_distribution.md" => guide_building_and_distribution_md,
|
||||||
"guide/src/building_and_distribution/multiple_python_versions.md" => guide_bnd_multiple_python_versions_md,
|
"guide/src/building_and_distribution/multiple_python_versions.md" => guide_bnd_multiple_python_versions_md,
|
||||||
"guide/src/class.md" => guide_class_md,
|
"guide/src/class.md" => guide_class_md,
|
||||||
|
|
|
@ -0,0 +1,98 @@
|
||||||
|
#![cfg(feature = "macros")]
|
||||||
|
#![cfg(not(target_arch = "wasm32"))]
|
||||||
|
use std::{task::Poll, thread, time::Duration};
|
||||||
|
|
||||||
|
use futures::{channel::oneshot, future::poll_fn};
|
||||||
|
use pyo3::{prelude::*, py_run};
|
||||||
|
|
||||||
|
#[path = "../src/tests/common.rs"]
|
||||||
|
mod common;
|
||||||
|
|
||||||
|
fn handle_windows(test: &str) -> String {
|
||||||
|
let set_event_loop_policy = r#"
|
||||||
|
import asyncio, sys
|
||||||
|
if sys.platform == "win32":
|
||||||
|
asyncio.set_event_loop_policy(asyncio.WindowsSelectorEventLoopPolicy())
|
||||||
|
"#;
|
||||||
|
pyo3::unindent::unindent(set_event_loop_policy) + &pyo3::unindent::unindent(test)
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn noop_coroutine() {
|
||||||
|
#[pyfunction]
|
||||||
|
async fn noop() -> usize {
|
||||||
|
42
|
||||||
|
}
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
let noop = wrap_pyfunction!(noop, gil).unwrap();
|
||||||
|
let test = "import asyncio; assert asyncio.run(noop()) == 42";
|
||||||
|
py_run!(gil, noop, &handle_windows(test));
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn sleep_0_like_coroutine() {
|
||||||
|
#[pyfunction]
|
||||||
|
async fn sleep_0() -> usize {
|
||||||
|
let mut waken = false;
|
||||||
|
poll_fn(|cx| {
|
||||||
|
if !waken {
|
||||||
|
cx.waker().wake_by_ref();
|
||||||
|
waken = true;
|
||||||
|
return Poll::Pending;
|
||||||
|
}
|
||||||
|
Poll::Ready(42)
|
||||||
|
})
|
||||||
|
.await
|
||||||
|
}
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
let sleep_0 = wrap_pyfunction!(sleep_0, gil).unwrap();
|
||||||
|
let test = "import asyncio; assert asyncio.run(sleep_0()) == 42";
|
||||||
|
py_run!(gil, sleep_0, &handle_windows(test));
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
#[pyfunction]
|
||||||
|
async fn sleep(seconds: f64) -> usize {
|
||||||
|
let (tx, rx) = oneshot::channel();
|
||||||
|
thread::spawn(move || {
|
||||||
|
thread::sleep(Duration::from_secs_f64(seconds));
|
||||||
|
tx.send(42).unwrap();
|
||||||
|
});
|
||||||
|
rx.await.unwrap()
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn sleep_coroutine() {
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
let sleep = wrap_pyfunction!(sleep, gil).unwrap();
|
||||||
|
let test = r#"import asyncio; assert asyncio.run(sleep(0.1)) == 42"#;
|
||||||
|
py_run!(gil, sleep, &handle_windows(test));
|
||||||
|
})
|
||||||
|
}
|
||||||
|
|
||||||
|
#[test]
|
||||||
|
fn cancelled_coroutine() {
|
||||||
|
Python::with_gil(|gil| {
|
||||||
|
let sleep = wrap_pyfunction!(sleep, gil).unwrap();
|
||||||
|
let test = r#"
|
||||||
|
import asyncio
|
||||||
|
async def main():
|
||||||
|
task = asyncio.create_task(sleep(1))
|
||||||
|
await asyncio.sleep(0)
|
||||||
|
task.cancel()
|
||||||
|
await task
|
||||||
|
asyncio.run(main())
|
||||||
|
"#;
|
||||||
|
let globals = gil.import("__main__").unwrap().dict();
|
||||||
|
globals.set_item("sleep", sleep).unwrap();
|
||||||
|
let err = gil
|
||||||
|
.run(
|
||||||
|
&pyo3::unindent::unindent(&handle_windows(test)),
|
||||||
|
Some(globals),
|
||||||
|
None,
|
||||||
|
)
|
||||||
|
.unwrap_err();
|
||||||
|
assert_eq!(err.value(gil).get_type().name().unwrap(), "CancelledError");
|
||||||
|
})
|
||||||
|
}
|
|
@ -4,6 +4,8 @@ error[E0277]: the trait bound `PyDict: PyClass` is not satisfied
|
||||||
5 | #[pyclass(extends=PyDict)]
|
5 | #[pyclass(extends=PyDict)]
|
||||||
| ^^^^^^^^^^^^^^^^^^^^^^^^^^ the trait `PyClass` is not implemented for `PyDict`
|
| ^^^^^^^^^^^^^^^^^^^^^^^^^^ the trait `PyClass` is not implemented for `PyDict`
|
||||||
|
|
|
|
||||||
= help: the trait `PyClass` is implemented for `TestClass`
|
= help: the following other types implement trait `PyClass`:
|
||||||
|
TestClass
|
||||||
|
Coroutine
|
||||||
= note: required for `PyDict` to implement `PyClassBaseType`
|
= note: required for `PyDict` to implement `PyClassBaseType`
|
||||||
= note: this error originates in the attribute macro `pyclass` (in Nightly builds, run with -Z macro-backtrace for more info)
|
= note: this error originates in the attribute macro `pyclass` (in Nightly builds, run with -Z macro-backtrace for more info)
|
||||||
|
|
|
@ -6,9 +6,6 @@ fn generic_function<T>(value: T) {}
|
||||||
#[pyfunction]
|
#[pyfunction]
|
||||||
fn impl_trait_function(impl_trait: impl AsRef<PyAny>) {}
|
fn impl_trait_function(impl_trait: impl AsRef<PyAny>) {}
|
||||||
|
|
||||||
#[pyfunction]
|
|
||||||
async fn async_function() {}
|
|
||||||
|
|
||||||
#[pyfunction]
|
#[pyfunction]
|
||||||
fn wildcard_argument(_: i32) {}
|
fn wildcard_argument(_: i32) {}
|
||||||
|
|
||||||
|
|
|
@ -10,29 +10,21 @@ error: Python functions cannot have `impl Trait` arguments
|
||||||
7 | fn impl_trait_function(impl_trait: impl AsRef<PyAny>) {}
|
7 | fn impl_trait_function(impl_trait: impl AsRef<PyAny>) {}
|
||||||
| ^^^^
|
| ^^^^
|
||||||
|
|
||||||
error: `async fn` is not yet supported for Python functions.
|
|
||||||
|
|
||||||
Additional crates such as `pyo3-asyncio` can be used to integrate async Rust and Python. For more information, see https://github.com/PyO3/pyo3/issues/1632
|
|
||||||
--> tests/ui/invalid_pyfunctions.rs:10:1
|
|
||||||
|
|
|
||||||
10 | async fn async_function() {}
|
|
||||||
| ^^^^^
|
|
||||||
|
|
||||||
error: wildcard argument names are not supported
|
error: wildcard argument names are not supported
|
||||||
--> tests/ui/invalid_pyfunctions.rs:13:22
|
--> tests/ui/invalid_pyfunctions.rs:10:22
|
||||||
|
|
|
|
||||||
13 | fn wildcard_argument(_: i32) {}
|
10 | fn wildcard_argument(_: i32) {}
|
||||||
| ^
|
| ^
|
||||||
|
|
||||||
error: destructuring in arguments is not supported
|
error: destructuring in arguments is not supported
|
||||||
--> tests/ui/invalid_pyfunctions.rs:16:26
|
--> tests/ui/invalid_pyfunctions.rs:13:26
|
||||||
|
|
|
|
||||||
16 | fn destructured_argument((a, b): (i32, i32)) {}
|
13 | fn destructured_argument((a, b): (i32, i32)) {}
|
||||||
| ^^^^^^
|
| ^^^^^^
|
||||||
|
|
||||||
error: required arguments after an `Option<_>` argument are ambiguous
|
error: required arguments after an `Option<_>` argument are ambiguous
|
||||||
= help: add a `#[pyo3(signature)]` annotation on this function to unambiguously specify the default values for all optional parameters
|
= help: add a `#[pyo3(signature)]` annotation on this function to unambiguously specify the default values for all optional parameters
|
||||||
--> tests/ui/invalid_pyfunctions.rs:19:63
|
--> tests/ui/invalid_pyfunctions.rs:16:63
|
||||||
|
|
|
|
||||||
19 | fn function_with_required_after_option(_opt: Option<i32>, _x: i32) {}
|
16 | fn function_with_required_after_option(_opt: Option<i32>, _x: i32) {}
|
||||||
| ^^^
|
| ^^^
|
||||||
|
|
|
@ -161,11 +161,6 @@ impl MyClass {
|
||||||
fn impl_trait_method_second_arg(&self, impl_trait: impl AsRef<PyAny>) {}
|
fn impl_trait_method_second_arg(&self, impl_trait: impl AsRef<PyAny>) {}
|
||||||
}
|
}
|
||||||
|
|
||||||
#[pymethods]
|
|
||||||
impl MyClass {
|
|
||||||
async fn async_method(&self) {}
|
|
||||||
}
|
|
||||||
|
|
||||||
#[pymethods]
|
#[pymethods]
|
||||||
impl MyClass {
|
impl MyClass {
|
||||||
#[pyo3(pass_module)]
|
#[pyo3(pass_module)]
|
||||||
|
|
|
@ -153,38 +153,30 @@ error: Python functions cannot have `impl Trait` arguments
|
||||||
161 | fn impl_trait_method_second_arg(&self, impl_trait: impl AsRef<PyAny>) {}
|
161 | fn impl_trait_method_second_arg(&self, impl_trait: impl AsRef<PyAny>) {}
|
||||||
| ^^^^
|
| ^^^^
|
||||||
|
|
||||||
error: `async fn` is not yet supported for Python functions.
|
|
||||||
|
|
||||||
Additional crates such as `pyo3-asyncio` can be used to integrate async Rust and Python. For more information, see https://github.com/PyO3/pyo3/issues/1632
|
|
||||||
--> tests/ui/invalid_pymethods.rs:166:5
|
|
||||||
|
|
|
||||||
166 | async fn async_method(&self) {}
|
|
||||||
| ^^^^^
|
|
||||||
|
|
||||||
error: `pass_module` cannot be used on Python methods
|
error: `pass_module` cannot be used on Python methods
|
||||||
--> tests/ui/invalid_pymethods.rs:171:12
|
--> tests/ui/invalid_pymethods.rs:166:12
|
||||||
|
|
|
|
||||||
171 | #[pyo3(pass_module)]
|
166 | #[pyo3(pass_module)]
|
||||||
| ^^^^^^^^^^^
|
| ^^^^^^^^^^^
|
||||||
|
|
||||||
error: Python objects are shared, so 'self' cannot be moved out of the Python interpreter.
|
error: Python objects are shared, so 'self' cannot be moved out of the Python interpreter.
|
||||||
Try `&self`, `&mut self, `slf: PyRef<'_, Self>` or `slf: PyRefMut<'_, Self>`.
|
Try `&self`, `&mut self, `slf: PyRef<'_, Self>` or `slf: PyRefMut<'_, Self>`.
|
||||||
--> tests/ui/invalid_pymethods.rs:177:29
|
--> tests/ui/invalid_pymethods.rs:172:29
|
||||||
|
|
|
|
||||||
177 | fn method_self_by_value(self) {}
|
172 | fn method_self_by_value(self) {}
|
||||||
| ^^^^
|
| ^^^^
|
||||||
|
|
||||||
error: macros cannot be used as items in `#[pymethods]` impl blocks
|
error: macros cannot be used as items in `#[pymethods]` impl blocks
|
||||||
= note: this was previously accepted and ignored
|
= note: this was previously accepted and ignored
|
||||||
--> tests/ui/invalid_pymethods.rs:212:5
|
--> tests/ui/invalid_pymethods.rs:207:5
|
||||||
|
|
|
|
||||||
212 | macro_invocation!();
|
207 | macro_invocation!();
|
||||||
| ^^^^^^^^^^^^^^^^
|
| ^^^^^^^^^^^^^^^^
|
||||||
|
|
||||||
error[E0119]: conflicting implementations of trait `pyo3::impl_::pyclass::PyClassNewTextSignature<TwoNew>` for type `pyo3::impl_::pyclass::PyClassImplCollector<TwoNew>`
|
error[E0119]: conflicting implementations of trait `pyo3::impl_::pyclass::PyClassNewTextSignature<TwoNew>` for type `pyo3::impl_::pyclass::PyClassImplCollector<TwoNew>`
|
||||||
--> tests/ui/invalid_pymethods.rs:182:1
|
--> tests/ui/invalid_pymethods.rs:177:1
|
||||||
|
|
|
|
||||||
182 | #[pymethods]
|
177 | #[pymethods]
|
||||||
| ^^^^^^^^^^^^
|
| ^^^^^^^^^^^^
|
||||||
| |
|
| |
|
||||||
| first implementation here
|
| first implementation here
|
||||||
|
@ -193,9 +185,9 @@ error[E0119]: conflicting implementations of trait `pyo3::impl_::pyclass::PyClas
|
||||||
= note: this error originates in the attribute macro `pymethods` (in Nightly builds, run with -Z macro-backtrace for more info)
|
= note: this error originates in the attribute macro `pymethods` (in Nightly builds, run with -Z macro-backtrace for more info)
|
||||||
|
|
||||||
error[E0592]: duplicate definitions with name `__pymethod___new____`
|
error[E0592]: duplicate definitions with name `__pymethod___new____`
|
||||||
--> tests/ui/invalid_pymethods.rs:182:1
|
--> tests/ui/invalid_pymethods.rs:177:1
|
||||||
|
|
|
|
||||||
182 | #[pymethods]
|
177 | #[pymethods]
|
||||||
| ^^^^^^^^^^^^
|
| ^^^^^^^^^^^^
|
||||||
| |
|
| |
|
||||||
| duplicate definitions for `__pymethod___new____`
|
| duplicate definitions for `__pymethod___new____`
|
||||||
|
@ -204,9 +196,9 @@ error[E0592]: duplicate definitions with name `__pymethod___new____`
|
||||||
= note: this error originates in the attribute macro `pymethods` (in Nightly builds, run with -Z macro-backtrace for more info)
|
= note: this error originates in the attribute macro `pymethods` (in Nightly builds, run with -Z macro-backtrace for more info)
|
||||||
|
|
||||||
error[E0592]: duplicate definitions with name `__pymethod_func__`
|
error[E0592]: duplicate definitions with name `__pymethod_func__`
|
||||||
--> tests/ui/invalid_pymethods.rs:197:1
|
--> tests/ui/invalid_pymethods.rs:192:1
|
||||||
|
|
|
|
||||||
197 | #[pymethods]
|
192 | #[pymethods]
|
||||||
| ^^^^^^^^^^^^
|
| ^^^^^^^^^^^^
|
||||||
| |
|
| |
|
||||||
| duplicate definitions for `__pymethod_func__`
|
| duplicate definitions for `__pymethod_func__`
|
||||||
|
|
Loading…
Reference in New Issue