feat(python): switched to a "Promise" approach

chore(python): removed, commented out some of the earlier attempts
feat(python): reworked the logger and tokio runtime approach, now the logging
callback drives the runtime.
This commit is contained in:
cschen 2024-08-20 21:09:10 +02:00
parent 72896d87d6
commit feff54bcdf
4 changed files with 170 additions and 152 deletions

24
dist/py/codemp.pyi vendored
View file

@ -1,14 +1,24 @@
from typing import Tuple, Optional from typing import Tuple, Optional, Callable
class PyLogger: class Driver:
""" """
A python wrapper for the tracing subscriber so that we can this is akin to a big red button with a white "STOP" on top of it.
receive logging messages from the library. it is used to stop the runtime
""" """
def __init__(self, debug) -> None: ... def stop(self) -> None: ...
async def listen(self) -> Optional[str]: ...
def init(logger_cb: Callable, debug: bool) -> Driver: ...
class RustPromise[T]:
"""
This is a class akin to a future, which wraps a join handle from a spawned
task on the rust side. you may call .pyawait() on this promise to block
until we have a result, or return immediately if we already have one.
This only goes one way rust -> python.
"""
def pyawait(self) -> T: ...
class TextChange: class TextChange:
""" """
Editor agnostic representation of a text change, it translate between internal Editor agnostic representation of a text change, it translate between internal
@ -84,7 +94,7 @@ class Client:
to a server and joining/creating new workspaces to a server and joining/creating new workspaces
""" """
def __new__(cls, host: str, username: str, password: str) -> None: ... def __new__(cls, host: str, username: str, password: str) -> None: ...
async def join_workspace(self, workspace: str) -> Workspace: ... def join_workspace(self, workspace: str) -> RustPromise: ...
def leave_workspace(self, workspace: str) -> bool: ... def leave_workspace(self, workspace: str) -> bool: ...
def get_workspace(self, id: str) -> Workspace: ... def get_workspace(self, id: str) -> Workspace: ...
def active_workspaces(self) -> list[str]: ... def active_workspaces(self) -> list[str]: ...

View file

@ -11,25 +11,18 @@ impl Client {
tokio().block_on(Client::new(host, username, password)) tokio().block_on(Client::new(host, username, password))
} }
async fn dioboia(&self) {
tokio().spawn(async { tracing::info!("dioboia? si dioboia!") });
}
#[pyo3(name = "join_workspace")] #[pyo3(name = "join_workspace")]
async fn pyjoin_workspace(&self, workspace: String) -> crate::Result<Workspace> { fn pyjoin_workspace(&self, workspace: String) -> PyResult<super::RustPromise> {
// self.join_workspace(workspace).await tracing::info!("attempting to join the workspace {workspace}");
// crate::a_sync! { self => self.join_workspace(workspace).await }
let rc = self.clone(); let rc = self.clone();
crate::spawn_future!(rc.join_workspace(workspace)) Ok(super::RustPromise(Some(tokio().spawn(async move {
.await Ok(rc
.unwrap() .join_workspace(workspace)
// This expands to if spawn_future_allow_threads! is used .await
// tokio() .map(|f| Python::with_gil(|py| f.into_py(py)))?)
// .spawn(super::AllowThreads(Box::pin(async move { }))))
// rc.join_workspace(workspace).await
// })))
// or if only spawn_future!
// tokio()
// .spawn(async move { rc.join_workspace(workspace).await })
} }
#[pyo3(name = "leave_workspace")] #[pyo3(name = "leave_workspace")]

View file

@ -2,13 +2,6 @@ pub mod client;
pub mod controllers; pub mod controllers;
pub mod workspace; pub mod workspace;
use std::{
future::Future,
pin::Pin,
sync::OnceLock,
task::{Context, Poll},
};
use crate::{ use crate::{
api::{Cursor, TextChange}, api::{Cursor, TextChange},
buffer::Controller as BufferController, buffer::Controller as BufferController,
@ -18,12 +11,14 @@ use crate::{
use pyo3::prelude::*; use pyo3::prelude::*;
use pyo3::{ use pyo3::{
exceptions::{PyConnectionError, PyRuntimeError, PySystemError}, exceptions::{PyConnectionError, PyRuntimeError, PySystemError},
ffi::PyFunctionObject,
types::PyFunction, types::PyFunction,
}; };
use tokio::sync::watch;
use tokio::sync::{mpsc, oneshot};
// global reference to a current_thread tokio runtime
pub fn tokio() -> &'static tokio::runtime::Runtime { pub fn tokio() -> &'static tokio::runtime::Runtime {
use std::sync::OnceLock;
static RT: OnceLock<tokio::runtime::Runtime> = OnceLock::new(); static RT: OnceLock<tokio::runtime::Runtime> = OnceLock::new();
RT.get_or_init(|| { RT.get_or_init(|| {
tokio::runtime::Builder::new_current_thread() tokio::runtime::Builder::new_current_thread()
@ -35,51 +30,151 @@ pub fn tokio() -> &'static tokio::runtime::Runtime {
}) })
} }
// workaround to allow the GIL to be released across awaits, waiting on // // workaround to allow the GIL to be released across awaits, waiting on
// https://github.com/PyO3/pyo3/pull/3610 // // https://github.com/PyO3/pyo3/pull/3610
struct AllowThreads<F>(F); // struct AllowThreads<F>(F);
impl<F> Future for AllowThreads<F> // impl<F> Future for AllowThreads<F>
where // where
F: Future + Unpin + Send, // F: Future + Unpin + Send,
F::Output: Send, // F::Output: Send,
{ // {
type Output = F::Output; // type Output = F::Output;
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> { // fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let waker = cx.waker(); // let waker = cx.waker();
let fut = unsafe { self.map_unchecked_mut(|e| &mut e.0) }; // let fut = unsafe { self.map_unchecked_mut(|e| &mut e.0) };
Python::with_gil(|py| py.allow_threads(|| fut.poll(&mut Context::from_waker(waker)))) // Python::with_gil(|py| py.allow_threads(|| fut.poll(&mut Context::from_waker(waker))))
} // }
} // }
// #[macro_export]
// macro_rules! spawn_future_allow_threads {
// ($fut:expr) => {
// $crate::ffi::python::tokio().spawn($crate::ffi::python::AllowThreads(Box::pin(
// async move {
// tracing::info!("running future from rust.");
// $fut.await
// },
// )))
// };
// }
#[macro_export] #[macro_export]
macro_rules! a_sync { macro_rules! a_sync {
($($clone:ident)* => $x:expr) => { ($($clone:ident)* => $x:expr) => {
{ {
$(let $clone = $clone.clone();)* $(let $clone = $clone.clone();)*
Ok(Promise(Some($crate::ffi::python::tokio().spawn(async move { $x })))) Ok($crate::ffi::python::RustPromise(Some($crate::ffi::python::tokio().spawn(async move {
Ok($x.map(|f| Python::with_gil(|py| f.into_py(py)))?)
}))))
} }
}; };
} }
#[macro_export] // #[macro_export]
macro_rules! spawn_future_allow_threads { // macro_rules! spawn_future {
($fut:expr) => { // ($fut:expr) => {
$crate::ffi::python::tokio().spawn($crate::ffi::python::AllowThreads(Box::pin( // $crate::ffi::python::tokio().spawn(async move { $fut.await })
async move { // };
tracing::info!("running future from rust."); // }
$fut.await
}, #[derive(Debug, Clone)]
))) struct LoggerProducer(mpsc::UnboundedSender<String>);
};
impl std::io::Write for LoggerProducer {
fn write(&mut self, buf: &[u8]) -> std::io::Result<usize> {
let _ = self.0.send(String::from_utf8_lossy(buf).to_string()); // ignore: logger disconnected or with full buffer
Ok(buf.len())
}
fn flush(&mut self) -> std::io::Result<()> {
Ok(())
}
} }
#[macro_export] #[pyclass]
macro_rules! spawn_future { pub struct Driver(Option<oneshot::Sender<()>>);
($fut:expr) => { #[pymethods]
$crate::ffi::python::tokio().spawn(async move { $fut.await }) impl Driver {
fn stop(&mut self) -> PyResult<()> {
match self.0.take() {
Some(tx) => {
let _ = tx.send(());
Ok(())
}
None => Err(PySystemError::new_err("Runtime was already stopped.")),
}
}
}
#[pyfunction]
fn init(logging_cb: Py<PyFunction>, debug: bool) -> PyResult<PyObject> {
let (tx, mut rx) = mpsc::unbounded_channel();
let level = if debug {
tracing::Level::DEBUG
} else {
tracing::Level::INFO
}; };
let format = tracing_subscriber::fmt::format()
.without_time()
.with_level(true)
.with_target(true)
.with_thread_ids(false)
.with_thread_names(false)
.with_file(false)
.with_line_number(false)
.with_source_location(false)
.compact();
let log_subscribing = tracing_subscriber::fmt()
.with_ansi(false)
.event_format(format)
.with_max_level(level)
.with_writer(std::sync::Mutex::new(LoggerProducer(tx)))
.try_init();
let (rt_stop_tx, rt_stop_rx) = oneshot::channel::<()>();
match log_subscribing {
Ok(_) => {
// the runtime is driven by the logger awaiting messages from codemp and echoing them back to
// python logger.
std::thread::spawn(move || {
tokio().block_on(async move {
tokio::select! {
biased;
Some(msg) = rx.recv() => {
let _ = Python::with_gil(|py| logging_cb.call1(py, (msg,)));
},
_ = rt_stop_rx => { todo!() },
}
})
});
Ok(Python::with_gil(|py| Driver(Some(rt_stop_tx)).into_py(py)))
}
Err(_) => Err(PyRuntimeError::new_err("codemp was already initialised.")),
}
}
#[pyclass]
pub struct RustPromise(Option<tokio::task::JoinHandle<PyResult<PyObject>>>);
#[pymethods]
impl RustPromise {
#[pyo3(name = "pyawait")]
fn _await(&mut self) -> PyResult<PyObject> {
match self.0.take() {
None => Err(PySystemError::new_err(
"promise can't be awaited multiple times!",
)),
Some(x) => match tokio().block_on(x) {
Err(e) => Err(PySystemError::new_err(format!(
"error awaiting promise: {e}"
))),
Ok(res) => res,
},
}
}
} }
impl From<crate::Error> for PyErr { impl From<crate::Error> for PyErr {
@ -105,90 +200,10 @@ impl IntoPy<PyObject> for crate::api::User {
} }
} }
#[derive(Debug, Clone)]
struct LoggerProducer(watch::Sender<String>);
impl std::io::Write for LoggerProducer {
fn write(&mut self, buf: &[u8]) -> std::io::Result<usize> {
let _ = self.0.send(String::from_utf8_lossy(buf).to_string()); // ignore: logger disconnected or with full buffer
Ok(buf.len())
}
fn flush(&mut self) -> std::io::Result<()> {
Ok(())
}
}
#[pyclass]
struct PyLogger(watch::Receiver<String>);
#[pymethods]
impl PyLogger {
#[new]
fn init_logger(debug: bool) -> PyResult<Self> {
let (tx, mut rx) = watch::channel("logger initialised".to_string());
let level = if debug {
tracing::Level::DEBUG
} else {
tracing::Level::INFO
};
let format = tracing_subscriber::fmt::format()
.without_time()
.with_level(true)
.with_target(true)
.with_thread_ids(false)
.with_thread_names(false)
.with_file(false)
.with_line_number(false)
.with_source_location(false)
.compact();
match tracing_subscriber::fmt()
.with_ansi(false)
.event_format(format)
.with_max_level(level)
.with_writer(std::sync::Mutex::new(LoggerProducer(tx)))
.try_init()
{
Ok(_) => Ok(PyLogger(rx)),
Err(_) => Err(PySystemError::new_err("A logger already exists")),
}
}
async fn listen(&mut self) -> Option<String> {
if self.0.changed().await.is_ok() {
return Some(self.0.borrow().clone());
}
None
}
}
#[pyclass]
pub struct Promise(Option<tokio::task::JoinHandle<PyResult<PyObject>>>);
#[pymethods]
impl Promise {
#[pyo3(name = "await")]
fn a_wait(&mut self) -> PyResult<PyObject> {
match self.0.take() {
None => Err(PySystemError::new_err(
"promise can't be awaited multiple times!",
)),
Some(x) => match tokio().block_on(x) {
Err(e) => Err(PySystemError::new_err(format!(
"error awaiting promise: {e}"
))),
Ok(res) => res,
},
}
}
}
#[pymodule] #[pymodule]
fn codemp(m: &Bound<'_, PyModule>) -> PyResult<()> { fn codemp(m: &Bound<'_, PyModule>) -> PyResult<()> {
m.add_class::<PyLogger>()?; m.add_function(wrap_pyfunction!(init, m)?)?;
m.add_class::<Driver>()?;
m.add_class::<TextChange>()?; m.add_class::<TextChange>()?;
m.add_class::<BufferController>()?; m.add_class::<BufferController>()?;

View file

@ -5,7 +5,7 @@ use pyo3::prelude::*;
use crate::spawn_future; use crate::spawn_future;
use super::Promise; // use super::Promise;
#[pymethods] #[pymethods]
impl Workspace { impl Workspace {
@ -31,12 +31,12 @@ impl Workspace {
} }
} }
#[pyo3(name = "event")] // #[pyo3(name = "event")]
fn pyevent(&self) -> Promise { // fn pyevent(&self) -> Promise {
crate::a_sync! { self => // crate::a_sync! { self =>
self.event().await // self.event().await
} // }
} // }
#[pyo3(name = "fetch_buffers")] #[pyo3(name = "fetch_buffers")]
async fn pyfetch_buffers(&self) -> crate::Result<()> { async fn pyfetch_buffers(&self) -> crate::Result<()> {