mirror of
https://github.com/hexedtech/codemp.git
synced 2024-11-22 15:24:48 +01:00
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:
parent
72896d87d6
commit
feff54bcdf
4 changed files with 170 additions and 152 deletions
24
dist/py/codemp.pyi
vendored
24
dist/py/codemp.pyi
vendored
|
@ -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]: ...
|
||||||
|
|
|
@ -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")]
|
||||||
|
|
|
@ -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>()?;
|
||||||
|
|
|
@ -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<()> {
|
||||||
|
|
Loading…
Reference in a new issue