Refactor async io support

This commit is contained in:
Nikolay Kim 2024-09-10 15:48:16 +05:00
parent db6d3a6e4c
commit fc83a887bc
12 changed files with 242 additions and 530 deletions

View file

@ -1,6 +1,6 @@
[package] [package]
name = "ntex-async-std" name = "ntex-async-std"
version = "0.5.0" version = "0.5.1"
authors = ["ntex contributors <team@ntex.rs>"] authors = ["ntex contributors <team@ntex.rs>"]
description = "async-std intergration for ntex framework" description = "async-std intergration for ntex framework"
keywords = ["network", "framework", "async", "futures"] keywords = ["network", "framework", "async", "futures"]
@ -17,7 +17,7 @@ path = "src/lib.rs"
[dependencies] [dependencies]
ntex-bytes = "0.1" ntex-bytes = "0.1"
ntex-io = "2.0" ntex-io = "2.5"
ntex-util = "2.0" ntex-util = "2.0"
log = "0.4" log = "0.4"
async-std = { version = "1", features = ["unstable"] } async-std = { version = "1", features = ["unstable"] }

View file

@ -1,17 +1,19 @@
use std::{any, cell::RefCell, future::Future, io, pin::Pin, task::Context, task::Poll}; use std::future::{poll_fn, Future};
use std::{any, cell::RefCell, io, pin::Pin, task::Context, task::Poll};
use async_std::io::{Read, Write}; use async_std::io::{Read, Write};
use ntex_bytes::{Buf, BufMut, BytesVec}; use ntex_bytes::{Buf, BufMut, BytesVec};
use ntex_io::{ use ntex_io::{types, Handle, IoStream, ReadContext, WriteContext, WriteStatus};
types, Handle, IoStream, ReadContext, ReadStatus, WriteContext, WriteStatus,
};
use ntex_util::{ready, time::sleep, time::Sleep}; use ntex_util::{ready, time::sleep, time::Sleep};
use crate::TcpStream; use crate::TcpStream;
impl IoStream for TcpStream { impl IoStream for TcpStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
async_std::task::spawn_local(ReadTask::new(self.clone(), read)); let mut rio = ReadTask(RefCell::new(self.clone()));
async_std::task::spawn_local(async move {
read.handle(&mut rio).await;
});
async_std::task::spawn_local(WriteTask::new(self.clone(), write)); async_std::task::spawn_local(WriteTask::new(self.clone(), write));
Some(Box::new(self)) Some(Box::new(self))
} }
@ -29,64 +31,17 @@ impl Handle for TcpStream {
} }
/// Read io task /// Read io task
struct ReadTask { struct ReadTask(RefCell<TcpStream>);
io: RefCell<TcpStream>,
state: ReadContext,
}
impl ReadTask { impl ntex_io::AsyncRead for ReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: TcpStream, state: ReadContext) -> Self { // read data from socket
Self { let result = poll_fn(|cx| {
state, let mut io = self.0.borrow_mut();
io: RefCell::new(io), poll_read_buf(Pin::new(&mut io.0), cx, &mut buf)
} })
} .await;
} (buf, result)
impl Future for ReadTask {
type Output = ();
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_ref();
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
this.state.with_buf(|buf, hw, lw| {
// read data from socket
let mut io = self.io.borrow_mut();
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(Pin::new(&mut io.0), cx, buf) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!("async-std stream is disconnected");
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!("async-std read task failed on io {:?}", err);
Poll::Ready(Err(err))
}
};
}
})
}
ReadStatus::Terminate => {
log::trace!("read task is instructed to shutdown");
Poll::Ready(())
}
}
} }
} }
@ -342,71 +297,27 @@ mod unixstream {
impl IoStream for UnixStream { impl IoStream for UnixStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
async_std::task::spawn_local(ReadTask::new(self.clone(), read)); let mut rio = ReadTask(RefCell::new(self.clone()));
async_std::task::spawn_local(async move {
read.handle(&mut rio).await;
});
async_std::task::spawn_local(WriteTask::new(self, write)); async_std::task::spawn_local(WriteTask::new(self, write));
None None
} }
} }
/// Read io task /// Read io task
struct ReadTask { struct ReadTask(RefCell<UnixStream>);
io: RefCell<UnixStream>,
state: ReadContext,
}
impl ReadTask { impl ntex_io::AsyncRead for ReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: UnixStream, state: ReadContext) -> Self { // read data from socket
Self { let result = poll_fn(|cx| {
state, let mut io = self.0.borrow_mut();
io: RefCell::new(io), poll_read_buf(Pin::new(&mut io.0), cx, &mut buf)
}
}
}
impl Future for ReadTask {
type Output = ();
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_ref();
this.state.with_buf(|buf, hw, lw| {
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
// read data from socket
let mut io = this.io.borrow_mut();
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(Pin::new(&mut io.0), cx, buf) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!("async-std stream is disconnected");
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!("read task failed on io {:?}", err);
Poll::Ready(Err(err))
}
};
}
}
ReadStatus::Terminate => {
log::trace!("read task is instructed to shutdown");
Poll::Ready(Ok(()))
}
}
}) })
.await;
(buf, result)
} }
} }

View file

@ -1,6 +1,6 @@
[package] [package]
name = "ntex-compio" name = "ntex-compio"
version = "0.1.1" version = "0.1.2"
authors = ["ntex contributors <team@ntex.rs>"] authors = ["ntex contributors <team@ntex.rs>"]
description = "compio runtime intergration for ntex framework" description = "compio runtime intergration for ntex framework"
keywords = ["network", "framework", "async", "futures"] keywords = ["network", "framework", "async", "futures"]
@ -18,7 +18,7 @@ path = "src/lib.rs"
[dependencies] [dependencies]
ntex-bytes = "0.1" ntex-bytes = "0.1"
ntex-io = "2.3" ntex-io = "2.5"
ntex-util = "2" ntex-util = "2"
log = "0.4" log = "0.4"
compio-net = "0.4.1" compio-net = "0.4.1"

View file

@ -4,18 +4,28 @@ use compio::buf::{BufResult, IoBuf, IoBufMut, SetBufInit};
use compio::io::{AsyncRead, AsyncWrite}; use compio::io::{AsyncRead, AsyncWrite};
use compio::net::TcpStream; use compio::net::TcpStream;
use ntex_bytes::{Buf, BufMut, BytesVec}; use ntex_bytes::{Buf, BufMut, BytesVec};
use ntex_io::{ use ntex_io::{types, Handle, IoStream, ReadContext, WriteContext, WriteStatus};
types, Handle, IoStream, ReadContext, ReadStatus, WriteContext, WriteStatus,
};
use ntex_util::{future::select, future::Either, time::sleep}; use ntex_util::{future::select, future::Either, time::sleep};
impl IoStream for crate::TcpStream { impl IoStream for crate::TcpStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
let mut io = self.0.clone(); let io = self.0.clone();
compio::runtime::spawn(async move { compio::runtime::spawn(async move {
run(&mut io, &read, write).await; let mut wr_io = io.clone();
let wr_task = compio::runtime::spawn(async move {
write_task(&mut wr_io, &write).await;
log::debug!("{} Write task is stopped", write.tag());
});
let mut io = ReadIo(io);
match io.close().await { read.handle(&mut io).await;
log::debug!("{} Read task is stopped", read.tag());
if !wr_task.is_finished() {
let _ = wr_task.await;
}
match io.0.close().await {
Ok(_) => log::debug!("{} Stream is closed", read.tag()), Ok(_) => log::debug!("{} Stream is closed", read.tag()),
Err(e) => log::error!("{} Stream is closed, {:?}", read.tag(), e), Err(e) => log::error!("{} Stream is closed, {:?}", read.tag(), e),
} }
@ -29,11 +39,24 @@ impl IoStream for crate::TcpStream {
#[cfg(unix)] #[cfg(unix)]
impl IoStream for crate::UnixStream { impl IoStream for crate::UnixStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
let mut io = self.0; let io = self.0;
compio::runtime::spawn(async move { compio::runtime::spawn(async move {
run(&mut io, &read, write).await; let mut wr_io = io.clone();
let wr_task = compio::runtime::spawn(async move {
write_task(&mut wr_io, &write).await;
log::debug!("{} Write task is stopped", write.tag());
});
match io.close().await { let mut io = ReadIo(io);
read.handle(&mut io).await;
log::debug!("{} Read task is stopped", read.tag());
if !wr_task.is_finished() {
let _ = wr_task.await;
}
match io.0.close().await {
Ok(_) => log::debug!("{} Unix stream is closed", read.tag()), Ok(_) => log::debug!("{} Unix stream is closed", read.tag()),
Err(e) => log::error!("{} Unix stream is closed, {:?}", read.tag(), e), Err(e) => log::error!("{} Unix stream is closed, {:?}", read.tag(), e),
} }
@ -88,71 +111,15 @@ impl SetBufInit for CompioBuf {
} }
} }
async fn run<T: AsyncRead + AsyncWrite + Clone + 'static>( struct ReadIo<T: AsyncRead>(T);
io: &mut T,
read: &ReadContext,
write: WriteContext,
) {
let mut wr_io = io.clone();
let wr_task = compio::runtime::spawn(async move {
write_task(&mut wr_io, &write).await;
log::debug!("{} Write task is stopped", write.tag());
});
read_task(io, read).await; impl<T> ntex_io::AsyncRead for ReadIo<T>
log::debug!("{} Read task is stopped", read.tag()); where
T: AsyncRead,
if !wr_task.is_finished() { {
let _ = wr_task.await; async fn read(&mut self, buf: BytesVec) -> (BytesVec, io::Result<usize>) {
} let BufResult(result, buf) = self.0.read(CompioBuf(buf)).await;
} (buf.0, result)
/// Read io task
async fn read_task<T: AsyncRead>(io: &mut T, state: &ReadContext) {
loop {
match state.ready().await {
ReadStatus::Ready => {
let result = state
.with_buf_async(|buf| async {
let BufResult(result, buf) =
match select(io.read(CompioBuf(buf)), state.wait_for_close())
.await
{
Either::Left(res) => res,
Either::Right(_) => return (Default::default(), Ok(1)),
};
match result {
Ok(n) => {
if n == 0 {
log::trace!(
"{}: Tcp stream is disconnected",
state.tag()
);
}
(buf.0, Ok(n))
}
Err(err) => {
log::trace!(
"{}: Read task failed on io {:?}",
state.tag(),
err
);
(buf.0, Err(err))
}
}
})
.await;
if result.is_ready() {
break;
}
}
ReadStatus::Terminate => {
log::trace!("{}: Read task is instructed to shutdown", state.tag());
break;
}
}
} }
} }

View file

@ -1,6 +1,6 @@
[package] [package]
name = "ntex-glommio" name = "ntex-glommio"
version = "0.5.0" version = "0.5.1"
authors = ["ntex contributors <team@ntex.rs>"] authors = ["ntex contributors <team@ntex.rs>"]
description = "glommio intergration for ntex framework" description = "glommio intergration for ntex framework"
keywords = ["network", "framework", "async", "futures"] keywords = ["network", "framework", "async", "futures"]
@ -17,7 +17,7 @@ path = "src/lib.rs"
[dependencies] [dependencies]
ntex-bytes = "0.1" ntex-bytes = "0.1"
ntex-io = "2.0" ntex-io = "2.5"
ntex-util = "2.0" ntex-util = "2.0"
futures-lite = "2.2" futures-lite = "2.2"
log = "0.4" log = "0.4"

View file

@ -13,7 +13,10 @@ use crate::net_impl::{TcpStream, UnixStream};
impl IoStream for TcpStream { impl IoStream for TcpStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
glommio::spawn_local(ReadTask::new(self.clone(), read)).detach(); let mut rio = ReadTask(self.clone());
glommio::spawn_local(async move {
read.handle(&mut rio).await;
});
glommio::spawn_local(WriteTask::new(self.clone(), write)).detach(); glommio::spawn_local(WriteTask::new(self.clone(), write)).detach();
Some(Box::new(self)) Some(Box::new(self))
} }
@ -21,7 +24,10 @@ impl IoStream for TcpStream {
impl IoStream for UnixStream { impl IoStream for UnixStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
glommio::spawn_local(UnixReadTask::new(self.clone(), read)).detach(); let mut rio = UnixReadTask(self.clone());
glommio::spawn_local(async move {
read.handle(&mut rio).await;
});
glommio::spawn_local(UnixWriteTask::new(self, write)).detach(); glommio::spawn_local(UnixWriteTask::new(self, write)).detach();
None None
} }
@ -39,64 +45,17 @@ impl Handle for TcpStream {
} }
/// Read io task /// Read io task
struct ReadTask { struct ReadTask(TcpStream);
io: TcpStream,
state: ReadContext,
}
impl ReadTask { impl ntex_io::AsyncRead for ReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: TcpStream, state: ReadContext) -> Self { // read data from socket
Self { io, state } let result = poll_fn(|cx| {
} let mut io = self.0.borrow_mut();
} poll_read_buf(Pin::new(&mut *io), cx, &mut buf)
impl Future for ReadTask {
type Output = ();
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_mut();
this.state.with_buf(|buf, hw, lw| {
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
// read data from socket
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(
Pin::new(&mut *this.io.0.borrow_mut()),
cx,
buf,
) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!("glommio stream is disconnected");
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!("read task failed on io {:?}", err);
Poll::Ready(Err(err))
}
};
}
}
ReadStatus::Terminate => {
log::trace!("read task is instructed to shutdown");
Poll::Ready(Ok(()))
}
}
}) })
.await;
(buf, result)
} }
} }
@ -359,64 +318,17 @@ pub fn poll_read_buf<T: AsyncRead>(
} }
/// Read io task /// Read io task
struct UnixReadTask { struct UnixReadTask(UnixStream);
io: UnixStream,
state: ReadContext,
}
impl UnixReadTask { impl ntex_io::AsyncRead for UnixReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: UnixStream, state: ReadContext) -> Self { // read data from socket
Self { io, state } let result = poll_fn(|cx| {
} let mut io = self.0.borrow_mut();
} poll_read_buf(Pin::new(&mut *io), cx, &mut buf)
impl Future for UnixReadTask {
type Output = ();
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_mut();
this.state.with_buf(|buf, hw, lw| {
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
// read data from socket
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(
Pin::new(&mut *this.io.0.borrow_mut()),
cx,
buf,
) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!("glommio stream is disconnected");
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!("read task failed on io {:?}", err);
Poll::Ready(Err(err))
}
};
}
}
ReadStatus::Terminate => {
log::trace!("read task is instructed to shutdown");
Poll::Ready(Ok(()))
}
}
}) })
.await;
(buf, result)
} }
} }

View file

@ -1,5 +1,9 @@
# Changes # Changes
## [2.5.0] - 2024-09-10
* Refactor async io support
## [2.3.1] - 2024-09-05 ## [2.3.1] - 2024-09-05
* Tune async io tasks support * Tune async io tasks support

View file

@ -1,6 +1,6 @@
[package] [package]
name = "ntex-io" name = "ntex-io"
version = "2.4.0" version = "2.5.0"
authors = ["ntex contributors <team@ntex.rs>"] authors = ["ntex contributors <team@ntex.rs>"]
description = "Utilities for encoding and decoding frames" description = "Utilities for encoding and decoding frames"
keywords = ["network", "framework", "async", "futures"] keywords = ["network", "framework", "async", "futures"]

View file

@ -1,5 +1,6 @@
//! Utilities for abstructing io streams //! Utilities for abstructing io streams
#![deny(rust_2018_idioms, unreachable_pub, missing_debug_implementations)] #![deny(rust_2018_idioms, unreachable_pub, missing_debug_implementations)]
#![allow(async_fn_in_trait)]
use std::{ use std::{
any::Any, any::TypeId, fmt, io as sio, io::Error as IoError, task::Context, task::Poll, any::Any, any::TypeId, fmt, io as sio, io::Error as IoError, task::Context, task::Poll,
@ -20,6 +21,7 @@ mod tasks;
mod timer; mod timer;
mod utils; mod utils;
use ntex_bytes::BytesVec;
use ntex_codec::{Decoder, Encoder}; use ntex_codec::{Decoder, Encoder};
use ntex_util::time::Millis; use ntex_util::time::Millis;
@ -43,6 +45,11 @@ pub enum ReadStatus {
Terminate, Terminate,
} }
#[doc(hidden)]
pub trait AsyncRead {
async fn read(&mut self, buf: BytesVec) -> (BytesVec, sio::Result<usize>);
}
/// Status for write task /// Status for write task
#[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)] #[derive(Copy, Clone, Debug, PartialEq, Eq, Hash)]
pub enum WriteStatus { pub enum WriteStatus {

View file

@ -1,8 +1,9 @@
use std::{future::poll_fn, future::Future, io, task::Context, task::Poll}; use std::{future::poll_fn, future::Future, io, task::Context, task::Poll};
use ntex_bytes::{BufMut, BytesVec, PoolRef}; use ntex_bytes::{BufMut, BytesVec, PoolRef};
use ntex_util::future::{select, Either};
use crate::{Flags, IoRef, ReadStatus, WriteStatus}; use crate::{AsyncRead, Flags, IoRef, ReadStatus, WriteStatus};
#[derive(Debug)] #[derive(Debug)]
/// Context for io read task /// Context for io read task
@ -149,115 +150,124 @@ impl ReadContext {
} }
/// Get read buffer (async) /// Get read buffer (async)
pub async fn with_buf_async<F, R>(&self, f: F) -> Poll<()> pub async fn handle<T>(&self, io: &mut T)
where where
F: FnOnce(BytesVec) -> R, T: AsyncRead,
R: Future<Output = (BytesVec, io::Result<usize>)>,
{ {
let inner = &self.0 .0; let inner = &self.0 .0;
// // we already pushed new data to read buffer, loop {
// // we have to wait for dispatcher to read data from buffer let result = poll_fn(|cx| self.0.filter().poll_read_ready(cx)).await;
// if inner.flags.get().is_read_buf_ready() { if result == ReadStatus::Terminate {
// ntex_util::task::yield_to().await; log::trace!("{}: Read task is instructed to shutdown", self.tag());
// } break;
}
let mut buf = if inner.flags.get().is_read_buf_ready() { let mut buf = if inner.flags.get().is_read_buf_ready() {
// read buffer is still not read by dispatcher // read buffer is still not read by dispatcher
// we cannot touch it // we cannot touch it
inner.pool.get().get_read_buf() inner.pool.get().get_read_buf()
} else { } else {
inner inner
.buffer .buffer
.get_read_source() .get_read_source()
.unwrap_or_else(|| inner.pool.get().get_read_buf()) .unwrap_or_else(|| inner.pool.get().get_read_buf())
}; };
// make sure we've got room // make sure we've got room
let (hw, lw) = self.0.memory_pool().read_params().unpack(); let (hw, lw) = self.0.memory_pool().read_params().unpack();
let remaining = buf.remaining_mut(); let remaining = buf.remaining_mut();
if remaining <= lw { if remaining <= lw {
buf.reserve(hw - remaining); buf.reserve(hw - remaining);
} }
let total = buf.len(); let total = buf.len();
// call provided callback // call provided callback
let (buf, result) = f(buf).await; let (buf, result) = match select(io.read(buf), self.wait_for_close()).await {
let total2 = buf.len(); Either::Left(res) => res,
let nbytes = if total2 > total { total2 - total } else { 0 }; Either::Right(_) => {
let total = total2; log::trace!("{}: Read io is closed, stop read task", self.tag());
break;
}
};
if let Some(mut first_buf) = inner.buffer.get_read_source() { // handle incoming data
first_buf.extend_from_slice(&buf); let total2 = buf.len();
inner.buffer.set_read_source(&self.0, first_buf); let nbytes = if total2 > total { total2 - total } else { 0 };
} else { let total = total2;
inner.buffer.set_read_source(&self.0, buf);
}
// handle buffer changes if let Some(mut first_buf) = inner.buffer.get_read_source() {
if nbytes > 0 { first_buf.extend_from_slice(&buf);
let filter = self.0.filter(); inner.buffer.set_read_source(&self.0, first_buf);
let res = match filter.process_read_buf(&self.0, &inner.buffer, 0, nbytes) { } else {
Ok(status) => { inner.buffer.set_read_source(&self.0, buf);
if status.nbytes > 0 { }
// check read back-pressure
if hw < inner.buffer.read_destination_size() { // handle buffer changes
log::trace!( if nbytes > 0 {
let filter = self.0.filter();
let res = match filter.process_read_buf(&self.0, &inner.buffer, 0, nbytes) {
Ok(status) => {
if status.nbytes > 0 {
// check read back-pressure
if hw < inner.buffer.read_destination_size() {
log::trace!(
"{}: Io read buffer is too large {}, enable read back-pressure", "{}: Io read buffer is too large {}, enable read back-pressure",
self.0.tag(), self.0.tag(),
total total
); );
inner.insert_flags(Flags::BUF_R_READY | Flags::BUF_R_FULL); inner.insert_flags(Flags::BUF_R_READY | Flags::BUF_R_FULL);
} else { } else {
inner.insert_flags(Flags::BUF_R_READY); inner.insert_flags(Flags::BUF_R_READY);
}
log::trace!(
"{}: New {} bytes available, wakeup dispatcher",
self.0.tag(),
nbytes
);
// dest buffer has new data, wake up dispatcher
inner.dispatch_task.wake();
} else if inner.flags.get().contains(Flags::RD_NOTIFY) {
// in case of "notify" we must wake up dispatch task
// if we read any data from source
inner.dispatch_task.wake();
} }
log::trace!(
"{}: New {} bytes available, wakeup dispatcher",
self.0.tag(),
nbytes
);
// dest buffer has new data, wake up dispatcher
inner.dispatch_task.wake();
} else if inner.flags.get().contains(Flags::RD_NOTIFY) {
// in case of "notify" we must wake up dispatch task
// if we read any data from source
inner.dispatch_task.wake();
}
// while reading, filter wrote some data // while reading, filter wrote some data
// in that case filters need to process write buffers // in that case filters need to process write buffers
// and potentialy wake write task // and potentialy wake write task
if status.need_write { if status.need_write {
filter.process_write_buf(&self.0, &inner.buffer, 0) filter.process_write_buf(&self.0, &inner.buffer, 0)
} else { } else {
Ok(()) Ok(())
}
} }
Err(err) => Err(err),
};
if let Err(err) = res {
inner.dispatch_task.wake();
inner.io_stopped(Some(err));
inner.insert_flags(Flags::BUF_R_READY);
} }
Err(err) => Err(err),
};
if let Err(err) = res {
inner.dispatch_task.wake();
inner.io_stopped(Some(err));
inner.insert_flags(Flags::BUF_R_READY);
} }
}
match result { match result {
Ok(n) => { Ok(0) => {
if n == 0 { log::trace!("{}: Tcp stream is disconnected", self.tag());
inner.io_stopped(None); inner.io_stopped(None);
Poll::Ready(()) break;
} else { }
Ok(_) => {
if inner.flags.get().contains(Flags::IO_STOPPING_FILTERS) { if inner.flags.get().contains(Flags::IO_STOPPING_FILTERS) {
shutdown_filters(&self.0); shutdown_filters(&self.0);
} }
Poll::Pending
} }
} Err(err) => {
Err(e) => { log::trace!("{}: Read task failed on io {:?}", self.tag(), err);
inner.io_stopped(Some(e)); inner.io_stopped(Some(err));
Poll::Ready(()) break;
}
} }
} }
} }

View file

@ -1,6 +1,6 @@
[package] [package]
name = "ntex-tokio" name = "ntex-tokio"
version = "0.5.1" version = "0.5.2"
authors = ["ntex contributors <team@ntex.rs>"] authors = ["ntex contributors <team@ntex.rs>"]
description = "tokio intergration for ntex framework" description = "tokio intergration for ntex framework"
keywords = ["network", "framework", "async", "futures"] keywords = ["network", "framework", "async", "futures"]
@ -17,7 +17,7 @@ path = "src/lib.rs"
[dependencies] [dependencies]
ntex-bytes = "0.1" ntex-bytes = "0.1"
ntex-io = "2.4" ntex-io = "2.5"
ntex-util = "2" ntex-util = "2"
log = "0.4" log = "0.4"
tokio = { version = "1", default-features = false, features = ["rt", "net", "sync", "signal"] } tokio = { version = "1", default-features = false, features = ["rt", "net", "sync", "signal"] }

View file

@ -1,10 +1,10 @@
use std::future::{poll_fn, Future};
use std::task::{Context, Poll}; use std::task::{Context, Poll};
use std::{any, cell::RefCell, cmp, future::Future, io, mem, pin::Pin, rc::Rc, rc::Weak}; use std::{any, cell::RefCell, cmp, io, mem, pin::Pin, rc::Rc, rc::Weak};
use ntex_bytes::{Buf, BufMut, BytesVec}; use ntex_bytes::{Buf, BufMut, BytesVec};
use ntex_io::{ use ntex_io::{
types, Filter, Handle, Io, IoBoxed, IoStream, ReadContext, ReadStatus, WriteContext, types, Filter, Handle, Io, IoBoxed, IoStream, ReadContext, WriteContext, WriteStatus,
WriteStatus,
}; };
use ntex_util::{ready, time::sleep, time::Millis, time::Sleep}; use ntex_util::{ready, time::sleep, time::Millis, time::Sleep};
use tokio::io::{AsyncRead, AsyncWrite, ReadBuf}; use tokio::io::{AsyncRead, AsyncWrite, ReadBuf};
@ -14,7 +14,10 @@ impl IoStream for crate::TcpStream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
let io = Rc::new(RefCell::new(self.0)); let io = Rc::new(RefCell::new(self.0));
tokio::task::spawn_local(ReadTask::new(io.clone(), read)); let mut rio = ReadTask(io.clone());
tokio::task::spawn_local(async move {
read.handle(&mut rio).await;
});
tokio::task::spawn_local(WriteTask::new(io.clone(), write)); tokio::task::spawn_local(WriteTask::new(io.clone(), write));
Some(Box::new(HandleWrapper(io))) Some(Box::new(HandleWrapper(io)))
} }
@ -36,67 +39,17 @@ impl Handle for HandleWrapper {
} }
/// Read io task /// Read io task
struct ReadTask { struct ReadTask(Rc<RefCell<TcpStream>>);
io: Rc<RefCell<TcpStream>>,
state: ReadContext,
}
impl ReadTask { impl ntex_io::AsyncRead for ReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: Rc<RefCell<TcpStream>>, state: ReadContext) -> Self { // read data from socket
Self { io, state } let result = poll_fn(|cx| {
} let mut io = self.0.borrow_mut();
} poll_read_buf(Pin::new(&mut *io), cx, &mut buf)
})
impl Future for ReadTask { .await;
type Output = (); (buf, result)
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_ref();
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
this.state.with_buf(|buf, hw, lw| {
// read data from socket
let mut io = this.io.borrow_mut();
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(Pin::new(&mut *io), cx, buf) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!(
"{}: Tcp stream is disconnected",
this.state.tag()
);
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!(
"{}: Read task failed on io {:?}",
this.state.tag(),
err
);
Poll::Ready(Err(err))
}
};
}
})
}
ReadStatus::Terminate => {
log::trace!("{}: Read task is instructed to shutdown", this.state.tag());
Poll::Ready(())
}
}
} }
} }
@ -472,78 +425,26 @@ mod unixstream {
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> { fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
let io = Rc::new(RefCell::new(self.0)); let io = Rc::new(RefCell::new(self.0));
tokio::task::spawn_local(ReadTask::new(io.clone(), read)); let mut rio = ReadTask(io.clone());
tokio::task::spawn_local(async move {
read.handle(&mut rio).await;
});
tokio::task::spawn_local(WriteTask::new(io, write)); tokio::task::spawn_local(WriteTask::new(io, write));
None None
} }
} }
/// Read io task struct ReadTask(Rc<RefCell<UnixStream>>);
struct ReadTask {
io: Rc<RefCell<UnixStream>>,
state: ReadContext,
}
impl ReadTask { impl ntex_io::AsyncRead for ReadTask {
/// Create new read io task async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
fn new(io: Rc<RefCell<UnixStream>>, state: ReadContext) -> Self { // read data from socket
Self { io, state } let result = poll_fn(|cx| {
} let mut io = self.0.borrow_mut();
} poll_read_buf(Pin::new(&mut *io), cx, &mut buf)
impl Future for ReadTask {
type Output = ();
fn poll(self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
let this = self.as_ref();
this.state.with_buf(|buf, hw, lw| {
match ready!(this.state.poll_ready(cx)) {
ReadStatus::Ready => {
// read data from socket
let mut io = this.io.borrow_mut();
loop {
// make sure we've got room
let remaining = buf.remaining_mut();
if remaining < lw {
buf.reserve(hw - remaining);
}
return match poll_read_buf(Pin::new(&mut *io), cx, buf) {
Poll::Pending => Poll::Pending,
Poll::Ready(Ok(n)) => {
if n == 0 {
log::trace!(
"{}: Tokio unix stream is disconnected",
this.state.tag()
);
Poll::Ready(Ok(()))
} else if buf.len() < hw {
continue;
} else {
Poll::Pending
}
}
Poll::Ready(Err(err)) => {
log::trace!(
"{}: Unix stream read task failed {:?}",
this.state.tag(),
err
);
Poll::Ready(Err(err))
}
};
}
}
ReadStatus::Terminate => {
log::trace!(
"{}: Read task is instructed to shutdown",
this.state.tag()
);
Poll::Ready(Ok(()))
}
}
}) })
.await;
(buf, result)
} }
} }