mirror of
https://github.com/ntex-rs/ntex.git
synced 2025-04-03 21:07:39 +03:00
Refactor async io support (#417)
This commit is contained in:
parent
db6d3a6e4c
commit
1d529fab3c
24 changed files with 872 additions and 2508 deletions
|
@ -1,12 +1,12 @@
|
|||
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, future::poll_fn, io, mem, pin::Pin, rc::Rc, rc::Weak};
|
||||
|
||||
use ntex_bytes::{Buf, BufMut, BytesVec};
|
||||
use ntex_io::{
|
||||
types, Filter, Handle, Io, IoBoxed, IoStream, ReadContext, ReadStatus, WriteContext,
|
||||
WriteStatus,
|
||||
types, Filter, Handle, Io, IoBoxed, IoStream, ReadContext, WriteContext,
|
||||
WriteContextBuf,
|
||||
};
|
||||
use ntex_util::{ready, time::sleep, time::Millis, time::Sleep};
|
||||
use ntex_util::{ready, time::Millis};
|
||||
use tokio::io::{AsyncRead, AsyncWrite, ReadBuf};
|
||||
use tokio::net::TcpStream;
|
||||
|
||||
|
@ -14,8 +14,14 @@ impl IoStream for crate::TcpStream {
|
|||
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
|
||||
let io = Rc::new(RefCell::new(self.0));
|
||||
|
||||
tokio::task::spawn_local(ReadTask::new(io.clone(), read));
|
||||
tokio::task::spawn_local(WriteTask::new(io.clone(), write));
|
||||
let mut rio = Read(io.clone());
|
||||
tokio::task::spawn_local(async move {
|
||||
read.handle(&mut rio).await;
|
||||
});
|
||||
let mut wio = Write(io.clone());
|
||||
tokio::task::spawn_local(async move {
|
||||
write.handle(&mut wio).await;
|
||||
});
|
||||
Some(Box::new(HandleWrapper(io)))
|
||||
}
|
||||
}
|
||||
|
@ -36,345 +42,149 @@ impl Handle for HandleWrapper {
|
|||
}
|
||||
|
||||
/// Read io task
|
||||
struct ReadTask {
|
||||
io: Rc<RefCell<TcpStream>>,
|
||||
state: ReadContext,
|
||||
}
|
||||
struct Read(Rc<RefCell<TcpStream>>);
|
||||
|
||||
impl ReadTask {
|
||||
/// Create new read io task
|
||||
fn new(io: Rc<RefCell<TcpStream>>, state: ReadContext) -> Self {
|
||||
Self { io, state }
|
||||
}
|
||||
}
|
||||
|
||||
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 = 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(())
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
enum IoWriteState {
|
||||
Processing(Option<Sleep>),
|
||||
Shutdown(Sleep, Shutdown),
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
enum Shutdown {
|
||||
None,
|
||||
Flushed,
|
||||
Stopping(u16),
|
||||
}
|
||||
|
||||
/// Write io task
|
||||
struct WriteTask {
|
||||
st: IoWriteState,
|
||||
io: Rc<RefCell<TcpStream>>,
|
||||
state: WriteContext,
|
||||
}
|
||||
|
||||
impl WriteTask {
|
||||
/// Create new write io task
|
||||
fn new(io: Rc<RefCell<TcpStream>>, state: WriteContext) -> Self {
|
||||
Self {
|
||||
io,
|
||||
state,
|
||||
st: IoWriteState::Processing(None),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Future for WriteTask {
|
||||
type Output = ();
|
||||
|
||||
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let this = self.as_mut().get_mut();
|
||||
|
||||
if this.state.poll_close(cx).is_ready() {
|
||||
return Poll::Ready(());
|
||||
}
|
||||
|
||||
match this.st {
|
||||
IoWriteState::Processing(ref mut delay) => {
|
||||
match ready!(this.state.poll_ready(cx)) {
|
||||
WriteStatus::Ready => {
|
||||
if let Some(delay) = delay {
|
||||
if delay.poll_elapsed(cx).is_ready() {
|
||||
this.state.close(Some(io::Error::new(
|
||||
io::ErrorKind::TimedOut,
|
||||
"Operation timedout",
|
||||
)));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
}
|
||||
|
||||
// flush io stream
|
||||
match ready!(this.state.with_buf(|buf| flush_io(
|
||||
&mut *this.io.borrow_mut(),
|
||||
buf,
|
||||
cx,
|
||||
&this.state
|
||||
))) {
|
||||
Ok(()) => Poll::Pending,
|
||||
Err(e) => {
|
||||
this.state.close(Some(e));
|
||||
Poll::Ready(())
|
||||
}
|
||||
}
|
||||
}
|
||||
WriteStatus::Timeout(time) => {
|
||||
log::trace!(
|
||||
"{}: Initiate timeout delay for {:?}",
|
||||
this.state.tag(),
|
||||
time
|
||||
);
|
||||
if delay.is_none() {
|
||||
*delay = Some(sleep(time));
|
||||
}
|
||||
self.poll(cx)
|
||||
}
|
||||
WriteStatus::Shutdown(time) => {
|
||||
log::trace!(
|
||||
"{}: Write task is instructed to shutdown",
|
||||
this.state.tag()
|
||||
);
|
||||
|
||||
let timeout = if let Some(delay) = delay.take() {
|
||||
delay
|
||||
impl ntex_io::AsyncRead for Read {
|
||||
#[inline]
|
||||
async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
|
||||
// read data from socket
|
||||
let result = poll_fn(|cx| {
|
||||
let mut n = 0;
|
||||
let mut io = self.0.borrow_mut();
|
||||
loop {
|
||||
return match poll_read_buf(Pin::new(&mut *io), cx, &mut buf)? {
|
||||
Poll::Pending => {
|
||||
if n > 0 {
|
||||
Poll::Ready(Ok(n))
|
||||
} else {
|
||||
sleep(time)
|
||||
};
|
||||
|
||||
this.st = IoWriteState::Shutdown(timeout, Shutdown::None);
|
||||
self.poll(cx)
|
||||
}
|
||||
WriteStatus::Terminate => {
|
||||
log::trace!(
|
||||
"{}: Write task is instructed to terminate",
|
||||
this.state.tag()
|
||||
);
|
||||
|
||||
if !matches!(
|
||||
this.io.borrow().linger(),
|
||||
Ok(Some(std::time::Duration::ZERO))
|
||||
) {
|
||||
// call shutdown to prevent flushing data on terminated Io. when
|
||||
// linger is set to zero, closing will reset the connection, so
|
||||
// shutdown is not neccessary.
|
||||
let _ = Pin::new(&mut *this.io.borrow_mut()).poll_shutdown(cx);
|
||||
Poll::Pending
|
||||
}
|
||||
this.state.close(None);
|
||||
Poll::Ready(())
|
||||
}
|
||||
}
|
||||
Poll::Ready(size) => {
|
||||
n += size;
|
||||
if n > 0 && buf.remaining_mut() > 0 {
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Ok(n))
|
||||
}
|
||||
};
|
||||
}
|
||||
IoWriteState::Shutdown(ref mut delay, ref mut st) => {
|
||||
// close WRITE side and wait for disconnect on read side.
|
||||
// use disconnect timeout, otherwise it could hang forever.
|
||||
loop {
|
||||
if this.state.poll_close(cx).is_ready() {
|
||||
return Poll::Ready(());
|
||||
}
|
||||
match st {
|
||||
Shutdown::None => {
|
||||
// flush write buffer
|
||||
let mut io = this.io.borrow_mut();
|
||||
match this
|
||||
.state
|
||||
.with_buf(|buf| flush_io(&mut *io, buf, cx, &this.state))
|
||||
{
|
||||
Poll::Ready(Ok(())) => {
|
||||
*st = Shutdown::Flushed;
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Err(err)) => {
|
||||
log::trace!(
|
||||
"{}: Write task is closed with err during flush, {:?}", this.state.tag(),
|
||||
err
|
||||
);
|
||||
this.state.close(Some(err));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
Poll::Pending => (),
|
||||
}
|
||||
}
|
||||
Shutdown::Flushed => {
|
||||
// shutdown WRITE side
|
||||
match Pin::new(&mut *this.io.borrow_mut()).poll_shutdown(cx) {
|
||||
Poll::Ready(Ok(_)) => {
|
||||
*st = Shutdown::Stopping(0);
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Err(e)) => {
|
||||
log::trace!(
|
||||
"{}: Write task is closed with err during shutdown",
|
||||
this.state.tag()
|
||||
);
|
||||
this.state.close(Some(e));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
}
|
||||
Shutdown::Stopping(ref mut count) => {
|
||||
// read until 0 or err
|
||||
let mut buf = [0u8; 512];
|
||||
loop {
|
||||
let mut read_buf = ReadBuf::new(&mut buf);
|
||||
match Pin::new(&mut *this.io.borrow_mut())
|
||||
.poll_read(cx, &mut read_buf)
|
||||
{
|
||||
Poll::Ready(Err(_)) | Poll::Ready(Ok(_))
|
||||
if read_buf.filled().is_empty() =>
|
||||
{
|
||||
this.state.close(None);
|
||||
log::trace!(
|
||||
"{}: Tokio write task is stopped",
|
||||
this.state.tag()
|
||||
);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
Poll::Pending => {
|
||||
*count += read_buf.filled().len() as u16;
|
||||
if *count > 4096 {
|
||||
log::trace!("{}: Tokio write task is stopped, too much input", this.state.tag());
|
||||
this.state.close(None);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
break;
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
})
|
||||
.await;
|
||||
|
||||
// disconnect timeout
|
||||
if delay.poll_elapsed(cx).is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
log::trace!("{}: Write task is stopped after delay", this.state.tag());
|
||||
this.state.close(None);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
}
|
||||
}
|
||||
(buf, result)
|
||||
}
|
||||
}
|
||||
|
||||
struct Write(Rc<RefCell<TcpStream>>);
|
||||
|
||||
impl ntex_io::AsyncWrite for Write {
|
||||
#[inline]
|
||||
async fn write(&mut self, buf: &mut WriteContextBuf) -> io::Result<()> {
|
||||
poll_fn(|cx| {
|
||||
if let Some(mut b) = buf.take() {
|
||||
let result = flush_io(&mut *self.0.borrow_mut(), &mut b, cx);
|
||||
buf.set(b);
|
||||
result
|
||||
} else {
|
||||
Poll::Ready(Ok(()))
|
||||
}
|
||||
})
|
||||
.await
|
||||
}
|
||||
|
||||
#[inline]
|
||||
async fn flush(&mut self) -> io::Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
#[inline]
|
||||
async fn shutdown(&mut self) -> io::Result<()> {
|
||||
poll_fn(|cx| Pin::new(&mut *self.0.borrow_mut()).poll_shutdown(cx)).await
|
||||
}
|
||||
}
|
||||
|
||||
pub fn poll_read_buf<T: AsyncRead>(
|
||||
io: Pin<&mut T>,
|
||||
cx: &mut Context<'_>,
|
||||
buf: &mut BytesVec,
|
||||
) -> Poll<io::Result<usize>> {
|
||||
let n = {
|
||||
let dst =
|
||||
unsafe { &mut *(buf.chunk_mut() as *mut _ as *mut [mem::MaybeUninit<u8>]) };
|
||||
let mut buf = ReadBuf::uninit(dst);
|
||||
let ptr = buf.filled().as_ptr();
|
||||
if io.poll_read(cx, &mut buf)?.is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
|
||||
// Ensure the pointer does not change from under us
|
||||
assert_eq!(ptr, buf.filled().as_ptr());
|
||||
buf.filled().len()
|
||||
};
|
||||
|
||||
// Safety: This is guaranteed to be the number of initialized (and read)
|
||||
// bytes due to the invariants provided by `ReadBuf::filled`.
|
||||
unsafe {
|
||||
buf.advance_mut(n);
|
||||
}
|
||||
|
||||
Poll::Ready(Ok(n))
|
||||
}
|
||||
|
||||
/// Flush write buffer to underlying I/O stream.
|
||||
pub(super) fn flush_io<T: AsyncRead + AsyncWrite + Unpin>(
|
||||
io: &mut T,
|
||||
buf: &mut Option<BytesVec>,
|
||||
buf: &mut BytesVec,
|
||||
cx: &mut Context<'_>,
|
||||
st: &WriteContext,
|
||||
) -> Poll<io::Result<()>> {
|
||||
if let Some(buf) = buf {
|
||||
let len = buf.len();
|
||||
let len = buf.len();
|
||||
|
||||
if len != 0 {
|
||||
// log::trace!("{}: Flushing framed transport: {:?}", st.tag(), buf.len());
|
||||
if len != 0 {
|
||||
// log::trace!("{}: Flushing framed transport: {:?}", st.tag(), buf.len());
|
||||
|
||||
let mut written = 0;
|
||||
let result = loop {
|
||||
break match Pin::new(&mut *io).poll_write(cx, &buf[written..]) {
|
||||
Poll::Ready(Ok(n)) => {
|
||||
if n == 0 {
|
||||
log::trace!(
|
||||
"{}: Disconnected during flush, written {}",
|
||||
st.tag(),
|
||||
written
|
||||
);
|
||||
Poll::Ready(Err(io::Error::new(
|
||||
io::ErrorKind::WriteZero,
|
||||
"failed to write frame to transport",
|
||||
)))
|
||||
let mut written = 0;
|
||||
let result = loop {
|
||||
break match Pin::new(&mut *io).poll_write(cx, &buf[written..]) {
|
||||
Poll::Ready(Ok(n)) => {
|
||||
if n == 0 {
|
||||
Poll::Ready(Err(io::Error::new(
|
||||
io::ErrorKind::WriteZero,
|
||||
"failed to write frame to transport",
|
||||
)))
|
||||
} else {
|
||||
written += n;
|
||||
if written == len {
|
||||
buf.clear();
|
||||
Poll::Ready(Ok(()))
|
||||
} else {
|
||||
written += n;
|
||||
if written == len {
|
||||
buf.clear();
|
||||
Poll::Ready(Ok(()))
|
||||
} else {
|
||||
continue;
|
||||
}
|
||||
continue;
|
||||
}
|
||||
}
|
||||
Poll::Pending => {
|
||||
// remove written data
|
||||
buf.advance(written);
|
||||
Poll::Pending
|
||||
}
|
||||
Poll::Ready(Err(e)) => {
|
||||
log::trace!("{}: Error during flush: {}", st.tag(), e);
|
||||
Poll::Ready(Err(e))
|
||||
}
|
||||
};
|
||||
};
|
||||
// log::trace!("{}: flushed {} bytes", st.tag(), written);
|
||||
|
||||
// flush
|
||||
return if written > 0 {
|
||||
match Pin::new(&mut *io).poll_flush(cx) {
|
||||
Poll::Ready(Ok(_)) => result,
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Err(e)) => {
|
||||
log::trace!("{}: Error during flush: {}", st.tag(), e);
|
||||
Poll::Ready(Err(e))
|
||||
}
|
||||
}
|
||||
} else {
|
||||
result
|
||||
Poll::Pending => {
|
||||
// remove written data
|
||||
buf.advance(written);
|
||||
Poll::Pending
|
||||
}
|
||||
Poll::Ready(Err(e)) => Poll::Ready(Err(e)),
|
||||
};
|
||||
};
|
||||
// log::trace!("{}: flushed {} bytes", st.tag(), written);
|
||||
|
||||
// flush
|
||||
if written > 0 {
|
||||
match Pin::new(&mut *io).poll_flush(cx) {
|
||||
Poll::Ready(Ok(_)) => result,
|
||||
Poll::Pending => Poll::Pending,
|
||||
Poll::Ready(Err(e)) => Poll::Ready(Err(e)),
|
||||
}
|
||||
} else {
|
||||
result
|
||||
}
|
||||
} else {
|
||||
Poll::Ready(Ok(()))
|
||||
}
|
||||
Poll::Ready(Ok(()))
|
||||
}
|
||||
|
||||
pub struct TokioIoBoxed(IoBoxed);
|
||||
|
@ -472,294 +282,77 @@ mod unixstream {
|
|||
fn start(self, read: ReadContext, write: WriteContext) -> Option<Box<dyn Handle>> {
|
||||
let io = Rc::new(RefCell::new(self.0));
|
||||
|
||||
tokio::task::spawn_local(ReadTask::new(io.clone(), read));
|
||||
tokio::task::spawn_local(WriteTask::new(io, write));
|
||||
let mut rio = Read(io.clone());
|
||||
tokio::task::spawn_local(async move {
|
||||
read.handle(&mut rio).await;
|
||||
});
|
||||
let mut wio = Write(io.clone());
|
||||
tokio::task::spawn_local(async move {
|
||||
write.handle(&mut wio).await;
|
||||
});
|
||||
None
|
||||
}
|
||||
}
|
||||
|
||||
/// Read io task
|
||||
struct ReadTask {
|
||||
io: Rc<RefCell<UnixStream>>,
|
||||
state: ReadContext,
|
||||
}
|
||||
struct Read(Rc<RefCell<UnixStream>>);
|
||||
|
||||
impl ReadTask {
|
||||
/// Create new read io task
|
||||
fn new(io: Rc<RefCell<UnixStream>>, state: ReadContext) -> Self {
|
||||
Self { io, state }
|
||||
}
|
||||
}
|
||||
|
||||
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);
|
||||
impl ntex_io::AsyncRead for Read {
|
||||
#[inline]
|
||||
async fn read(&mut self, mut buf: BytesVec) -> (BytesVec, io::Result<usize>) {
|
||||
// read data from socket
|
||||
let result = poll_fn(|cx| {
|
||||
let mut n = 0;
|
||||
let mut io = self.0.borrow_mut();
|
||||
loop {
|
||||
return match poll_read_buf(Pin::new(&mut *io), cx, &mut buf)? {
|
||||
Poll::Pending => {
|
||||
if n > 0 {
|
||||
Poll::Ready(Ok(n))
|
||||
} else {
|
||||
Poll::Pending
|
||||
}
|
||||
|
||||
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(()))
|
||||
}
|
||||
Poll::Ready(size) => {
|
||||
n += size;
|
||||
if n > 0 && buf.remaining_mut() > 0 {
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Ok(n))
|
||||
}
|
||||
};
|
||||
}
|
||||
})
|
||||
.await;
|
||||
|
||||
(buf, result)
|
||||
}
|
||||
}
|
||||
|
||||
/// Write io task
|
||||
struct WriteTask {
|
||||
st: IoWriteState,
|
||||
io: Rc<RefCell<UnixStream>>,
|
||||
state: WriteContext,
|
||||
}
|
||||
struct Write(Rc<RefCell<UnixStream>>);
|
||||
|
||||
impl WriteTask {
|
||||
/// Create new write io task
|
||||
fn new(io: Rc<RefCell<UnixStream>>, state: WriteContext) -> Self {
|
||||
Self {
|
||||
io,
|
||||
state,
|
||||
st: IoWriteState::Processing(None),
|
||||
}
|
||||
impl ntex_io::AsyncWrite for Write {
|
||||
#[inline]
|
||||
async fn write(&mut self, buf: &mut WriteContextBuf) -> io::Result<()> {
|
||||
poll_fn(|cx| {
|
||||
if let Some(mut b) = buf.take() {
|
||||
let result = flush_io(&mut *self.0.borrow_mut(), &mut b, cx);
|
||||
buf.set(b);
|
||||
result
|
||||
} else {
|
||||
Poll::Ready(Ok(()))
|
||||
}
|
||||
})
|
||||
.await
|
||||
}
|
||||
}
|
||||
|
||||
impl Future for WriteTask {
|
||||
type Output = ();
|
||||
#[inline]
|
||||
async fn flush(&mut self) -> io::Result<()> {
|
||||
Ok(())
|
||||
}
|
||||
|
||||
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
let this = self.as_mut().get_mut();
|
||||
|
||||
if this.state.poll_close(cx).is_ready() {
|
||||
return Poll::Ready(());
|
||||
}
|
||||
|
||||
match this.st {
|
||||
IoWriteState::Processing(ref mut delay) => {
|
||||
match this.state.poll_ready(cx) {
|
||||
Poll::Ready(WriteStatus::Ready) => {
|
||||
if let Some(delay) = delay {
|
||||
if delay.poll_elapsed(cx).is_ready() {
|
||||
this.state.close(Some(io::Error::new(
|
||||
io::ErrorKind::TimedOut,
|
||||
"Operation timedout",
|
||||
)));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
}
|
||||
|
||||
// flush io stream
|
||||
match ready!(this.state.with_buf(|buf| flush_io(
|
||||
&mut *this.io.borrow_mut(),
|
||||
buf,
|
||||
cx,
|
||||
&this.state
|
||||
))) {
|
||||
Ok(()) => Poll::Pending,
|
||||
Err(e) => {
|
||||
this.state.close(Some(e));
|
||||
Poll::Ready(())
|
||||
}
|
||||
}
|
||||
}
|
||||
Poll::Ready(WriteStatus::Timeout(time)) => {
|
||||
if delay.is_none() {
|
||||
*delay = Some(sleep(time));
|
||||
}
|
||||
self.poll(cx)
|
||||
}
|
||||
Poll::Ready(WriteStatus::Shutdown(time)) => {
|
||||
log::trace!(
|
||||
"{}: Write task is instructed to shutdown",
|
||||
this.state.tag()
|
||||
);
|
||||
|
||||
let timeout = if let Some(delay) = delay.take() {
|
||||
delay
|
||||
} else {
|
||||
sleep(time)
|
||||
};
|
||||
|
||||
this.st = IoWriteState::Shutdown(timeout, Shutdown::None);
|
||||
self.poll(cx)
|
||||
}
|
||||
Poll::Ready(WriteStatus::Terminate) => {
|
||||
log::trace!(
|
||||
"{}: Write task is instructed to terminate",
|
||||
this.state.tag()
|
||||
);
|
||||
|
||||
let _ = Pin::new(&mut *this.io.borrow_mut()).poll_shutdown(cx);
|
||||
this.state.close(None);
|
||||
Poll::Ready(())
|
||||
}
|
||||
Poll::Pending => Poll::Pending,
|
||||
}
|
||||
}
|
||||
IoWriteState::Shutdown(ref mut delay, ref mut st) => {
|
||||
// close WRITE side and wait for disconnect on read side.
|
||||
// use disconnect timeout, otherwise it could hang forever.
|
||||
loop {
|
||||
if this.state.poll_close(cx).is_ready() {
|
||||
return Poll::Ready(());
|
||||
}
|
||||
match st {
|
||||
Shutdown::None => {
|
||||
// flush write buffer
|
||||
let mut io = this.io.borrow_mut();
|
||||
match this.state.with_buf(|buf| {
|
||||
flush_io(&mut *io, buf, cx, &this.state)
|
||||
}) {
|
||||
Poll::Ready(Ok(())) => {
|
||||
*st = Shutdown::Flushed;
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Err(err)) => {
|
||||
log::trace!(
|
||||
"{}: Write task is closed with err during flush, {:?}", this.state.tag(),
|
||||
err
|
||||
);
|
||||
this.state.close(Some(err));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
Poll::Pending => (),
|
||||
}
|
||||
}
|
||||
Shutdown::Flushed => {
|
||||
// shutdown WRITE side
|
||||
match Pin::new(&mut *this.io.borrow_mut()).poll_shutdown(cx)
|
||||
{
|
||||
Poll::Ready(Ok(_)) => {
|
||||
*st = Shutdown::Stopping(0);
|
||||
continue;
|
||||
}
|
||||
Poll::Ready(Err(e)) => {
|
||||
log::trace!(
|
||||
"{}: Write task is closed with err during shutdown", this.state.tag()
|
||||
);
|
||||
this.state.close(Some(e));
|
||||
return Poll::Ready(());
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
}
|
||||
Shutdown::Stopping(ref mut count) => {
|
||||
// read until 0 or err
|
||||
let mut buf = [0u8; 512];
|
||||
loop {
|
||||
let mut read_buf = ReadBuf::new(&mut buf);
|
||||
match Pin::new(&mut *this.io.borrow_mut())
|
||||
.poll_read(cx, &mut read_buf)
|
||||
{
|
||||
Poll::Ready(Err(_)) | Poll::Ready(Ok(_))
|
||||
if read_buf.filled().is_empty() =>
|
||||
{
|
||||
this.state.close(None);
|
||||
log::trace!(
|
||||
"{}: Write task is stopped",
|
||||
this.state.tag()
|
||||
);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
Poll::Pending => {
|
||||
*count += read_buf.filled().len() as u16;
|
||||
if *count > 4096 {
|
||||
log::trace!(
|
||||
"{}: Write task is stopped, too much input", this.state.tag()
|
||||
);
|
||||
this.state.close(None);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
break;
|
||||
}
|
||||
_ => (),
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// disconnect timeout
|
||||
if delay.poll_elapsed(cx).is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
log::trace!(
|
||||
"{}: Write task is stopped after delay",
|
||||
this.state.tag()
|
||||
);
|
||||
this.state.close(None);
|
||||
return Poll::Ready(());
|
||||
}
|
||||
}
|
||||
}
|
||||
#[inline]
|
||||
async fn shutdown(&mut self) -> io::Result<()> {
|
||||
poll_fn(|cx| Pin::new(&mut *self.0.borrow_mut()).poll_shutdown(cx)).await
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
pub fn poll_read_buf<T: AsyncRead>(
|
||||
io: Pin<&mut T>,
|
||||
cx: &mut Context<'_>,
|
||||
buf: &mut BytesVec,
|
||||
) -> Poll<io::Result<usize>> {
|
||||
let n = {
|
||||
let dst =
|
||||
unsafe { &mut *(buf.chunk_mut() as *mut _ as *mut [mem::MaybeUninit<u8>]) };
|
||||
let mut buf = ReadBuf::uninit(dst);
|
||||
let ptr = buf.filled().as_ptr();
|
||||
if io.poll_read(cx, &mut buf)?.is_pending() {
|
||||
return Poll::Pending;
|
||||
}
|
||||
|
||||
// Ensure the pointer does not change from under us
|
||||
assert_eq!(ptr, buf.filled().as_ptr());
|
||||
buf.filled().len()
|
||||
};
|
||||
|
||||
// Safety: This is guaranteed to be the number of initialized (and read)
|
||||
// bytes due to the invariants provided by `ReadBuf::filled`.
|
||||
unsafe {
|
||||
buf.advance_mut(n);
|
||||
}
|
||||
|
||||
Poll::Ready(Ok(n))
|
||||
}
|
||||
|
|
|
@ -4,10 +4,8 @@ use ntex_bytes::PoolRef;
|
|||
use ntex_io::Io;
|
||||
|
||||
mod io;
|
||||
mod signals;
|
||||
|
||||
pub use self::io::{SocketOptions, TokioIoBoxed};
|
||||
pub use self::signals::{signal, Signal};
|
||||
|
||||
struct TcpStream(tokio::net::TcpStream);
|
||||
|
||||
|
|
|
@ -1,138 +0,0 @@
|
|||
use std::{
|
||||
cell::RefCell, future::Future, mem, pin::Pin, rc::Rc, task::Context, task::Poll,
|
||||
};
|
||||
|
||||
use tokio::sync::oneshot;
|
||||
use tokio::task::spawn_local;
|
||||
|
||||
thread_local! {
|
||||
static SRUN: RefCell<bool> = const { RefCell::new(false) };
|
||||
static SHANDLERS: Rc<RefCell<Vec<oneshot::Sender<Signal>>>> = Default::default();
|
||||
}
|
||||
|
||||
/// Different types of process signals
|
||||
#[derive(PartialEq, Eq, Clone, Copy, Debug)]
|
||||
pub enum Signal {
|
||||
/// SIGHUP
|
||||
Hup,
|
||||
/// SIGINT
|
||||
Int,
|
||||
/// SIGTERM
|
||||
Term,
|
||||
/// SIGQUIT
|
||||
Quit,
|
||||
}
|
||||
|
||||
/// Register signal handler.
|
||||
///
|
||||
/// Signals are handled by oneshots, you have to re-register
|
||||
/// after each signal.
|
||||
pub fn signal() -> Option<oneshot::Receiver<Signal>> {
|
||||
if !SRUN.with(|v| *v.borrow()) {
|
||||
spawn_local(Signals::new());
|
||||
}
|
||||
SHANDLERS.with(|handlers| {
|
||||
let (tx, rx) = oneshot::channel();
|
||||
handlers.borrow_mut().push(tx);
|
||||
Some(rx)
|
||||
})
|
||||
}
|
||||
|
||||
struct Signals {
|
||||
#[cfg(not(unix))]
|
||||
signal: Pin<Box<dyn Future<Output = std::io::Result<()>>>>,
|
||||
#[cfg(unix)]
|
||||
signals: Vec<(
|
||||
Signal,
|
||||
tokio::signal::unix::Signal,
|
||||
tokio::signal::unix::SignalKind,
|
||||
)>,
|
||||
}
|
||||
|
||||
impl Signals {
|
||||
fn new() -> Signals {
|
||||
SRUN.with(|h| *h.borrow_mut() = true);
|
||||
|
||||
#[cfg(not(unix))]
|
||||
{
|
||||
Signals {
|
||||
signal: Box::pin(tokio::signal::ctrl_c()),
|
||||
}
|
||||
}
|
||||
|
||||
#[cfg(unix)]
|
||||
{
|
||||
use tokio::signal::unix;
|
||||
|
||||
let sig_map = [
|
||||
(unix::SignalKind::interrupt(), Signal::Int),
|
||||
(unix::SignalKind::hangup(), Signal::Hup),
|
||||
(unix::SignalKind::terminate(), Signal::Term),
|
||||
(unix::SignalKind::quit(), Signal::Quit),
|
||||
];
|
||||
|
||||
let mut signals = Vec::new();
|
||||
for (kind, sig) in sig_map.iter() {
|
||||
match unix::signal(*kind) {
|
||||
Ok(stream) => signals.push((*sig, stream, *kind)),
|
||||
Err(e) => log::error!(
|
||||
"Cannot initialize stream handler for {:?} err: {}",
|
||||
sig,
|
||||
e
|
||||
),
|
||||
}
|
||||
}
|
||||
|
||||
Signals { signals }
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl Drop for Signals {
|
||||
fn drop(&mut self) {
|
||||
SRUN.with(|h| *h.borrow_mut() = false);
|
||||
}
|
||||
}
|
||||
|
||||
impl Future for Signals {
|
||||
type Output = ();
|
||||
|
||||
fn poll(mut self: Pin<&mut Self>, cx: &mut Context<'_>) -> Poll<Self::Output> {
|
||||
#[cfg(not(unix))]
|
||||
{
|
||||
if self.signal.as_mut().poll(cx).is_ready() {
|
||||
let handlers = SHANDLERS.with(|h| mem::take(&mut *h.borrow_mut()));
|
||||
for sender in handlers {
|
||||
let _ = sender.send(Signal::Int);
|
||||
}
|
||||
}
|
||||
Poll::Pending
|
||||
}
|
||||
#[cfg(unix)]
|
||||
{
|
||||
for (sig, stream, kind) in self.signals.iter_mut() {
|
||||
loop {
|
||||
if Pin::new(&mut *stream).poll_recv(cx).is_ready() {
|
||||
let handlers = SHANDLERS.with(|h| mem::take(&mut *h.borrow_mut()));
|
||||
for sender in handlers {
|
||||
let _ = sender.send(*sig);
|
||||
}
|
||||
match tokio::signal::unix::signal(*kind) {
|
||||
Ok(s) => {
|
||||
*stream = s;
|
||||
continue;
|
||||
}
|
||||
Err(e) => log::error!(
|
||||
"Cannot initialize stream handler for {:?} err: {}",
|
||||
sig,
|
||||
e
|
||||
),
|
||||
}
|
||||
}
|
||||
break;
|
||||
}
|
||||
}
|
||||
Poll::Pending
|
||||
}
|
||||
}
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue