Git Product home page Git Product logo

monoio's Introduction

Monoio

A thread-per-core Rust runtime with io_uring/epoll/kqueue.

Crates.io MIT/Apache-2 licensed Build Status Codecov FOSSA Status 中文说明

Design Goal

Monoio is a pure io_uring/epoll/kqueue Rust async runtime. Part of the design has been borrowed from Tokio and Tokio-uring. However, unlike Tokio-uring, Monoio does not run on top of another runtime, rendering it more efficient.

Moreover, Monoio is designed with a thread-per-core model in mind. Users do not need to worry about tasks being Send or Sync, as thread local storage can be used safely. In other words, the data does not escape the thread on await points, unlike on work-stealing runtimes such as Tokio. This is because for some use cases, specifically those targeted by this runtime, it is not necessary to make task schedulable between threads. For example, if we were to write a load balancer like NGINX, we would write it in a thread-per-core way. The thread local data does not need to be shared between threads, so the Sync and Send do not need to be implemented in the first place.

As you may have guessed, this runtime is primarily targeted at servers, where operations are io-bound on network sockets, and therefore the use of native asynchronous I/O APIs maximizes the throughput of the server. In order for Monoio to be as efficient as possible, we've enabled some unstable Rust features, and we've designed a whole new IO abstraction, which unfortunately may cause some compatibility problems. Our benchmarks prove that, for our use-cases, Monoio has a better performance than other Rust runtimes.

Quick Start

To use monoio, you need rust 1.75. If you already installed it, please make sure it is the latest version.

Also, if you want to use io_uring, you must make sure your kernel supports it(5.6+). And, memlock is configured as a proper number. If your kernel version does not meet the requirements, you can try to use the legacy driver to start, currently supports Linux and macOS(ref here).

🚧Experimental windows support is on the way.

Here is a basic example of how to use Monoio.

/// A echo example.
///
/// Run the example and `nc 127.0.0.1 50002` in another shell.
/// All your input will be echoed out.
use monoio::io::{AsyncReadRent, AsyncWriteRentExt};
use monoio::net::{TcpListener, TcpStream};

#[monoio::main]
async fn main() {
    let listener = TcpListener::bind("127.0.0.1:50002").unwrap();
    println!("listening");
    loop {
        let incoming = listener.accept().await;
        match incoming {
            Ok((stream, addr)) => {
                println!("accepted a connection from {}", addr);
                monoio::spawn(echo(stream));
            }
            Err(e) => {
                println!("accepted connection failed: {}", e);
                return;
            }
        }
    }
}

async fn echo(mut stream: TcpStream) -> std::io::Result<()> {
    let mut buf: Vec<u8> = Vec::with_capacity(8 * 1024);
    let mut res;
    loop {
        // read
        (res, buf) = stream.read(buf).await;
        if res? == 0 {
            return Ok(());
        }

        // write all
        (res, buf) = stream.write_all(buf).await;
        res?;

        // clear
        buf.clear();
    }
}

You can find more example code in examples of this repository.

Limitations

  1. On Linux 5.6 or newer, Monoio can use uring or epoll as io driver. On lower versions of Linux, it can only run in epoll mode. On macOS, kqueue can be used. Other platforms are currently not supported.
  2. Monoio can not solve all problems. If the workload is very unbalanced, it may cause performance degradation than Tokio since CPU cores may not be fully utilized.

Contributors

Thanks for their contributions!

Community

Monoio is a subproject of CloudWeGo. We are committed to building a cloud native ecosystem.

Associated Projects

HTTP framework and RPC framework are on the way.

Licenses

Monoio is licensed under the MIT license or Apache license.

During developing we referenced a lot from Tokio, Mio, Tokio-uring and other related projects. We would like to thank the authors of these projects.

FOSSA Status

monoio's People

Contributors

18o avatar auuo avatar bobozhengsir avatar carrotzrule123 avatar cssivision avatar dependabot[bot] avatar dragonly avatar fossabot avatar har23k avatar hxzhao527 avatar ihciah avatar import-yuefeng avatar jon-chuang avatar kingtous avatar lemonhx avatar liuxin231 avatar lizhanhui avatar loongs-zhang avatar lorislibralato avatar lz1998 avatar neetdai avatar pickfire avatar pymongo avatar reloginn avatar schrodingerzhu avatar songzhi avatar thewawar avatar tshepang avatar youjiali1995 avatar zzau13 avatar

Stargazers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

Watchers

 avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar  avatar

monoio's Issues

want RDMA supports

Is your feature request related to a problem? Please describe.
A clear and concise description of what the problem is. Ex. I'm always frustrated when [...]

Describe the solution you'd like
A clear and concise description of what you want to happen.

Describe alternatives you've considered
A clear and concise description of any alternative solutions or features you've considered.

Additional context
Add any other context or screenshots about the feature request here.

关于适用场景

贵项目文档中说明:“如果负载并非常不均衡,相比公平调度模型的Tokio它可能会性能变差”,按照我的理解,多路复用能够在单核实现最好的性能,如果要利用现代机器多核,最好是每个核上使用这种模式,理论上这种模式性能很高。这里负载不均衡的场景是指什么呢?

panics: run example 'join'

Version
List the versions of all monoio crates you are using. The easiest way to get
this information is using cargo tree subcommand:

cargo tree | grep monoio

monoio v0.0.3 (github.com/bytedance/monoio/monoio)
├── monoio-macros v0.0.2 (proc-macro) (github.com/bytedance/monoio/monoio-macros)
monoio-compat v0.0.4 (github.com/bytedance/monoio/monoio-compat)
├── monoio v0.0.3 (github.com/bytedance/monoio/monoio) (*)
└── monoio v0.0.3 (github.com/bytedance/monoio/monoio) (*)
monoio-examples v0.0.0 (github.com/bytedance/monoio/examples)
├── monoio v0.0.3 (github.com/bytedance/monoio/monoio) (*)
├── monoio-compat v0.0.4 (github.com/bytedance/monoio/monoio-compat) (*)
monoio-macros v0.0.2 (proc-macro) (github.com/bytedance/monoio/monoio-macros) (*)

Platform
The output of uname -a and ulimit -l.

uname -a
Linux zephyrus 5.10.60.1-microsoft-standard-WSL2 #1 SMP Wed Aug 25 23:20:18 UTC 2021 x86_64 GNU/Linux
ulimit -l
64

Description
Enter your issue details here.
One way to structure the description:

[short summary of the bug]

RUST_BACKTRACE=1 cargo run --color=always --example join

Output:

thread 'main' panicked at 'Unable to build runtime.: Os { code: 12, kind: OutOfMemory, message: "Cannot allocate memory" }', github.com/bytedance/monoio/monoio/src/lib.rs:81:10
stack backtrace:
   0: rust_begin_unwind
             at /rustc/936238a92b2f9d6e7afe7dda69b4afd903f96399/library/std/src/panicking.rs:498:5
   1: core::panicking::panic_fmt
             at /rustc/936238a92b2f9d6e7afe7dda69b4afd903f96399/library/core/src/panicking.rs:106:14
   2: core::result::unwrap_failed
             at /rustc/936238a92b2f9d6e7afe7dda69b4afd903f96399/library/core/src/result.rs:1613:5
   3: core::result::Result<T,E>::expect
             at /rustc/936238a92b2f9d6e7afe7dda69b4afd903f96399/library/core/src/result.rs:1255:23
   4: monoio::start
             at ./monoio/src/lib.rs:79:18
   5: join::main
             at ./examples/join.rs:6:5
   6: core::ops::function::FnOnce::call_once
             at /rustc/936238a92b2f9d6e7afe7dda69b4afd903f96399/library/core/src/ops/function.rs:227:5
note: Some details are omitted, run with `RUST_BACKTRACE=full` for a verbose backtrace.

`async_std::task::sleep` hang the runtime indefinitely

Version

[dependencies]
async-std = "1.11.0"
monoio = { version = "0.0.6", features = [
  "async-cancel",
  "bytes",
  "iouring",
  "legacy",
  "macros",
  "utils",
  "debug",
] }

Platform
The output of uname -a and ulimit -l.

Linux Carbon 5.13.0-52-generic #59~20.04.1-Ubuntu SMP Thu Jun 16 21:21:28 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux
65536

Description
async_std::task::sleep hang the runtime indefinitely
One way to structure the description:

It seems that the runtime awaits the task::sleep indefinitely from async-std. I understand that this is not the recommended way of sleeping, however, I'm using some crates that happen to sleep in this manner and I can't retrofit monoio within it.

I tried this code:

use std::time::Duration;

fn main() -> () {
    let mut rt = monoio::RuntimeBuilder::<monoio::FusionDriver>::new()
        .enable_timer()
        .build()
        .unwrap();
    rt.block_on(async move {
        async_std::task::sleep(Duration::from_millis(200)).await;
        println!("async_std::task::sleep worked!")
    })
}

I expected to see this happen: Print "async_std::task::sleep worked!")

Instead, this happened: Never ending loop

I might not have used the right monoio configuration but I tried several runtime and did not see any difference.

Thanks in advance.

For your information, the package I'm trying to run is zenoh which use async-std heavily

Interface to write from multiple buffers in order

Is your feature request related to a problem? Please describe.
Currently, writing from multiple buffers in order requires awaiting first's completion before writing the second.

One should be able to pass a collection of IoBufs (Vec, &'static/Boxed slice) and monoio ought to queue their writes in order.

The alternative is copying data which is undesirable.

Allow user specify thread number for main macro

Is your feature request related to a problem? Please describe.
Currently main macro will only start a single thread. If I want to run with multi threads, I have to create thread and create runtime manually. It will be more convinent if user can set thread count on macro.

Describe the solution you'd like
Allow user to set thread count in main and test macro.

Additional context
Assigned to ihciah.

quic udp?

implement for quic udp with letsencrypt automatic cert?

error below : protocol error for h2-server when trying to run in browser

error below : protocol error for h2-server when trying to run in browser

cargo run --example h2-server --release
    Finished release [optimized] target(s) in 0.08s
     Running `target/release/examples/h2-server`
listening on Ok(0.0.0.0:8080)
listening on Ok(0.0.0.0:8080)
listening on Ok(0.0.0.0:8080)
listening on Ok(0.0.0.0:8080)
  -> err=Error { kind: GoAway(b"", PROTOCOL_ERROR, Library) }
  -> err=Error { kind: GoAway(b"", PROTOCOL_ERROR, Library) }
  -> err=Error { kind: GoAway(b"", PROTOCOL_ERROR, Library) }
  -> err=Error { kind: GoAway(b"", PROTOCOL_ERROR, Library) }

Latest nightly fails

Version
List the versions of all monoio crates you are using. The easiest way to get
this information is using cargo tree subcommand:

cargo tree | grep monoio

└── monoio v0.0.7 (https://github.com/bytedance/monoio?branch=master#06327ebc)
    ├── monoio-macros v0.0.3 (proc-macro) (https://github.com/bytedance/monoio?branch=master#06327ebc)

Platform
The output of uname -a and ulimit -l.

catsnow:~/monoioshit$ uname -a
Linux catsnow 5.18.0-2-amd64 #1 SMP PREEMPT_DYNAMIC Debian 5.18.5-1 (2022-06-16) x86_64 GNU/Linux
catsnow:~/monoioshit$ ulimit -l
4109686

Description
Enter your issue details here.
One way to structure the description:

Type inference fails.

Assuming the summary

[short summary of the bug]

I tried this code:

[minimum code that reproduces the bug]

Download your repo, or include latest git version in a new cargo project.

cargo +nightly-2022-07-20 check this fails

cargo +nightly-2022-07-19 check this passes

I expected to see this happen: [explanation]

No type inference bugs

Instead, this happened: [explanation]

error[E0277]: the trait bound `T: IoBufMut` is not satisfied
  --> /home/user/.cargo/git/checkouts/monoio-91d866d0e012895e/06327eb/monoio/src/io/util/prefixed_io.rs:45:9
   |
45 | /         async move {
46 | |             if buf.bytes_total() == 0 {
47 | |                 return (Ok(0), buf);
48 | |             }
...  |
68 | |             self.io.read(buf).await
69 | |         }
   | |_________^ the trait `IoBufMut` is not implemented for `T`
   |
note: required by a bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::read`
  --> /home/user/.cargo/git/checkouts/monoio-91d866d0e012895e/06327eb/monoio/src/io/util/prefixed_io.rs:44:16
   |
44 |     fn read<T: crate::buf::IoBufMut>(&mut self, mut buf: T) -> Self::ReadFuture<'_, T> {
   |                ^^^^^^^^^^^^^^^^^^^^ required by this bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::read`
help: consider further restricting this bound
   |
38 |         T: 'a + buf::io_buf::IoBufMut, Self: 'a;
   |               +++++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBufMut` is not satisfied
  --> /home/user/.cargo/git/checkouts/monoio-91d866d0e012895e/06327eb/monoio/src/io/util/prefixed_io.rs:73:9
   |
73 | /         async move {
74 | |             let slice = match IoVecWrapperMut::new(buf) {
75 | |                 Ok(slice) => slice,
76 | |                 Err(buf) => return (Ok(0), buf),
...  |
84 | |             (result, buf)
85 | |         }
   | |_________^ the trait `IoVecBufMut` is not implemented for `T`
   |
note: required by a bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::readv`
  --> /home/user/.cargo/git/checkouts/monoio-91d866d0e012895e/06327eb/monoio/src/io/util/prefixed_io.rs:72:17
   |
72 |     fn readv<T: crate::buf::IoVecBufMut>(&mut self, mut buf: T) -> Self::ReadvFuture<'_, T> {
   |                 ^^^^^^^^^^^^^^^^^^^^^^^ required by this bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::readv`
help: consider further restricting this bound
   |
42 |         T: 'a + buf::io_vec_buf::IoVecBufMut, Self: 'a;
   |               ++++++++++++++++++++++++++++++

cargo test meet two error

Version
monoio v0.0.3

Platform
Linux n227-074-054 4.14.81.bm.15-amd64 #1 SMP Debian 4.14.81.bm.15 Sun Sep 8 05:02:31 UTC 2019 x86_64 GNU/Linux

Description
when i run cargo test, there are two errors.

➜  monoio git:(master) cargo test
   Compiling monoio v0.0.3 (/data00/home/yuyanglong/code/rust/yuyanglong/other-projects/monoio/monoio)
error[E0271]: type mismatch resolving `<impl futures::Future<Output = [async output]> as futures::Future>::Output == ()`
  --> monoio/src/stream/stream_ext.rs:44:30
   |
34 |           async move {
   |  ____________________-
35 | |             while let Some(item) = self.next().await {
36 | |                 (f)(item);
37 | |             }
38 | |         }
   | |_________- the found `async` block
...
44 |   type ForEachFut<T, Fut, F> = impl Future<Output = ()>;
   |                                ^^^^^^^^^^^^^^^^^^^^^^^^ expected `()`, found associated type
   |
  ::: /data00/home/yuyanglong/.rustup/toolchains/nightly-x86_64-unknown-linux-gnu/lib/rustlib/src/rust/library/core/src/future/mod.rs:65:43
   |
65 |   pub const fn from_generator<T>(gen: T) -> impl Future<Output = T::Return>
   |                                             ------------------------------- the found opaque type
   |
   = note:    expected unit type `()`
           found associated type `<impl futures::Future<Output = [async output]> as futures::Future>::Output`
   = help: consider constraining the associated type `<impl futures::Future<Output = [async output]> as futures::Future>::Output` to `()`
   = note: for more information, visit https://doc.rust-lang.org/book/ch19-03-advanced-traits.html

error[E0277]: the trait bound `T: stream::Stream` is not satisfied
  --> monoio/src/stream/stream_ext.rs:44:30
   |
44 | type ForEachFut<T, Fut, F> = impl Future<Output = ()>;
   |                              ^^^^^^^^^^^^^^^^^^^^^^^^ the trait `stream::Stream` is not implemented for `T`
   |
help: consider restricting type parameter `T`
   |
44 | type ForEachFut<T: stream::Stream, Fut, F> = impl Future<Output = ()>;
   |                  ++++++++++++++++

Some errors have detailed explanations: E0271, E0277.
For more information about an error, try `rustc --explain E0271`.
error: could not compile `monoio` due to 2 previous errors
warning: build failed, waiting for other jobs to finish...
error: build failed

Implement spawn_blocking

Is your feature request related to a problem? Please describe.
We need spawn_blocking like tokio, or the latency will be very big if users do heavy calculation.

Describe the solution you'd like
Implement with an thread pool.

Describe alternatives you've considered
Maybe users can do "spawn" by themselves since they can do across-thread communication.

Additional context
Nop.

block_on from local runtime, like tokio

I need to be able to fetch the current runtime of each thread and call blocking methods.
How does tokio with the functions

I think I could implement it.

Edit:

I see that it is quite difficult to get a local Executor because it depends on the Driver trait. I'm doing tests removing everything that is not ui-ring. It may fork you. With this I come to say that there is no rush with the issue and thank you for your work. I have already seen that you have it reflected in #8

add monoio::signal::ctrl_c() support under feature "signal"

Is your feature request related to a problem? Please describe.
I am trying to use tokio::signal::ctrl_c() in monoio runtime, but got the error

thread 'main' panicked at 'there is no reactor running, must be called from the context of a Tokio 1.x runtime', /home/yliu/.cargo/registry/src/github.com-1ecc6299db9ec823/tokio-1.26.0/src/signal/unix.rs:499:5

Describe the solution you'd like
add monoio::signal::ctrl_c() support in feature "signal"

Describe alternatives you've considered
no alternative

Additional context
N/A

tracing tools for monoio, thread-per-core

tokio tracing is slow as requires Sync and uses slow primitives
Can take inspiration from https://github.com/tikv/minitrace-rust
And optimize further for thread-per-core.

Actually minitrace is already very thread-per-core friendly as it does thread-local span collection before batching them out to central collector.

So maybe we should use minitrace instead.

Context on how slow tokio/tracing is: https://www.youtube.com/watch?v=8xTaxC1RcXE

minitrace is probably fast enough for spans deep in the async runtime itself. We can set tracing level via features (off by default)

[features]
minitrace_level_1 = []
minitrace_level_2 = [ "minitrace_level_1" ]
minitrace_level_3 = [ "minitrace_level_2" ]

Unsafety of pinned buffer?

According to the docs, the runtime will pin the buffer of an IoBufMut. But doesn't this mean that the impl of AsyncReadRent for [u8; N] is unsound because it returns the buffer by move, which violates the pinning guarantees?

wake_by_ref() does not wake the executor

Version
0.0.9

Platform
OS: Ubuntu 20.4
Kernel version: 5.13.0

Description

I want to implement yield in tokio

I tried this code:

struct Yield {
    yielded: bool,
}

impl Future for Yield {
    type Output = ();

    fn poll(mut self: Pin<&mut Self>, ctx: &mut Context<'_>) -> Poll<()> {
        if self.yielded {
            Poll::Ready(())
        } else {
            self.yielded = true;
            ctx.waker().wake_by_ref();
            Poll::Pending
        }
    }
}

fn main(){
    let mut rt = RuntimeBuilder::<IoUringDriver>::new()
        .with_entries(1024)
        .build()
        .unwrap();
    rt.block_on(async move{
        let yield_now = Yield { yielded: false };
        yield_now.await;
    });
}

I expected to see this happen: runtime finishes the yield task

Instead, this happened: Code runs forever

AFAIK, according to the contract of the Waker, at least one poll should be called. BTW, I tested the code with tokio, it works fine

Update: If I enable the sync feature, the problem goes away. The problem happens in the block_on function of Runtime. If we do not enable the sync feature, the driver will go to park and tick the CQ. It seems that it ignores the dummy waker

Code coverage and unit test

Is your feature request related to a problem? Please describe.
Add code coverage and more unit tests.

Describe the solution you'd like
With instrument-coverage andgrcov.

Describe alternatives you've considered

Additional context

Implement poll_* for AsyncReadRent and AsyncWriteRent

Is your feature request related to a problem? Please describe.
I'm building an HTTP/2 server on top of monoio. The issue is that TLS crates like rustls and boring require std::io::Read and std::io::Write traits to be implemented.

Describe the solution you'd like
tokio_boring solves this issue by implementing aforementioned traits on a wrapper struct.
This solution requires poll_* methods to be present.

Create axum server example.

Is your feature request related to a problem? Please describe.
A clear and concise description of what the problem is. Ex. I'm always frustrated when [...]

Create an example using axum::Server with custom runtime and method routers. Axum is a very good abstraction for servers. It has routers, extractors etc, which are convenient. However, when using it with monoio it is not trivial because of a bunch of reasons.

use axum::routing::*;
use axum::Router;
// use hyper::service::Service;
use hyper::{server::conn::Http, service::service_fn};
use monoio::net::TcpListener;
use monoio_compat::TcpStreamCompat;
use std::future::Future;
use std::net::SocketAddr;

#[derive(Clone)]
struct HyperExecutor;

impl<F> hyper::rt::Executor<F> for HyperExecutor
where
    F: Future + 'static,
    F::Output: 'static,
{
    fn execute(&self, fut: F) {
        monoio::spawn(fut);
    }
}


async fn root() -> &'static str {
    "hi there"
}

#[monoio::main(threads = 4)]
async fn main() {
    let mut app = Router::new()
        .route("/", get(root))
        .into_make_service_with_connect_info::<SocketAddr>();

    // let sv = app.as_service();

    println!("Running http server on 0.0.0.0:8000");
    let addr = ([0, 0, 0, 0], 8000);

    let listener = TcpListener::bind(addr.into()).unwrap();

    loop {
        let (stream, _) = listener.accept().await.unwrap();
        let stream = unsafe { TcpStreamCompat::new(stream) };
        monoio::spawn(
            Http::new()
                .with_executor(HyperExecutor)
                .serve_connection(stream, app), // error here, trait mismatch
        );
    }
}

I tried using make_service_fn from hyper, but it takes AddrStream instead of TcpStream, unfortunately AddrStream has no public constructor.

Describe the solution you'd like
A clear and concise description of what you want to happen.

In hyper_server example,

let _ = serve_http(([0, 0, 0, 0], 23300), hyper_handler).await;

we should be able to get

let app = Router::new().route("/", get(root)).into_make_service();
serve_http(..., app).await;

basically axum handlers.

Additional context
Add any other context or screenshots about the feature request here.

If we can't manage to get this done, we can talk to hyper/axum maintainers to allow for some abstractions for different kinds of executors.

cargo +nightly bug

Version

# rustc -vV
rustc 1.65.0-nightly (015a824f2 2022-08-22)
binary: rustc
commit-hash: 015a824f2dffe32707fceb59c47effaf7b73486c
commit-date: 2022-08-22
host: x86_64-unknown-linux-gnu
release: 1.65.0-nightly
LLVM version: 15.0.0

monoio

root@ubuntu:~/proxy# cargo tree | grep monoio
└── monoio v0.0.6
    ├── monoio-macros v0.0.3 (proc-macro)

Platform

The output of uname -a and ulimit -l.

Description

root@ubuntu:~/proxy# cargo +nightly  check
    Checking futures-sink v0.3.23
    Checking futures-io v0.3.23
    Checking cfg-if v1.0.0
    Checking pin-project-lite v0.2.9
    Checking pin-utils v0.1.0
    Checking bitflags v1.3.2
    Checking byteorder v1.4.3
    Checking scoped-tls v1.0.0
    Checking bytes v1.2.1
    Checking fxhash v0.2.1
    Checking libc v0.2.132
    Checking futures-core v0.3.23
    Checking memchr v2.5.0
    Checking futures-task v0.3.23
    Checking log v0.4.17
    Checking slab v0.4.7
    Checking memoffset v0.6.5
    Checking io-uring v0.5.3
    Checking os_socketaddr v0.2.1
    Checking socket2 v0.4.4
    Checking futures-channel v0.3.23
    Checking mio v0.8.4
    Checking nix v0.24.2
    Checking futures-util v0.3.23
    Checking futures-executor v0.3.23
    Checking futures v0.3.23
    Checking monoio v0.0.6
error[E0277]: the trait bound `B: IoBufMut` is not satisfied
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent.rs:72:9
   |
72 |         async move { (Ok(amt), buf) }
   |         ^^^^^^^^^^^^^^^^^^^^^^^^^^^^^ the trait `IoBufMut` is not implemented for `B`
   |
note: required by a bound in `<&[u8] as AsyncReadRent>::read`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent.rs:64:16
   |
64 |     fn read<T: IoBufMut>(&mut self, mut buf: T) -> Self::ReadFuture<'_, T> {
   |                ^^^^^^^^ required by this bound in `<&[u8] as AsyncReadRent>::read`
help: consider further restricting this bound
   |
60 |         B: 'a + buf::io_buf::IoBufMut, Self: 'a;
   |               +++++++++++++++++++++++

error[E0277]: the trait bound `B: IoVecBufMut` is not satisfied
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent.rs:76:9
   |
76 | /         async move {
77 | |             let n = match unsafe { RawBuf::new_from_iovec_mut(&mut buf) } {
78 | |                 Some(raw_buf) => self.read(raw_buf).await.0,
79 | |                 None => Ok(0),
...  |
84 | |             (n, buf)
85 | |         }
   | |_________^ the trait `IoVecBufMut` is not implemented for `B`
   |
note: required by a bound in `<&[u8] as AsyncReadRent>::readv`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent.rs:75:17
   |
75 |     fn readv<T: IoVecBufMut>(&mut self, mut buf: T) -> Self::ReadvFuture<'_, T> {
   |                 ^^^^^^^^^^^ required by this bound in `<&[u8] as AsyncReadRent>::readv`
help: consider further restricting this bound
   |
62 |         B: 'a + buf::io_vec_buf::IoVecBufMut, Self: 'a;
   |               ++++++++++++++++++++++++++++++

error[E0277]: the trait bound `T: IoBufMut` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent_ext.rs:109:9
    |
109 | /         async move {
110 | |             let ptr = buf.write_ptr();
111 | |             let len = buf.bytes_total();
112 | |             let mut read = 0;
...   |
133 | |             (Ok(read), buf)
134 | |         }
    | |_________^ the trait `IoBufMut` is not implemented for `T`
    |
note: required by a bound in `<A as AsyncReadRentExt>::read_exact`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent_ext.rs:107:22
    |
105 |     fn read_exact<T>(&mut self, mut buf: T) -> Self::ReadExactFuture<'_, T>
    |        ---------- required by a bound in this
106 |     where
107 |         T: 'static + IoBufMut,
    |                      ^^^^^^^^ required by this bound in `<A as AsyncReadRentExt>::read_exact`
help: consider further restricting this bound
    |
103 |     type ReadExactFuture<'a, T> = impl Future<Output = BufResult<usize, T>> where A: 'a, T: 'a + buf::io_buf::IoBufMut;
    |                                                                                                +++++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBufMut` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent_ext.rs:149:9
    |
149 | /         async move {
150 | |             while read < len {
151 | |                 let (res, meta_) = self.readv(meta).await;
152 | |                 meta = meta_;
...   |
168 | |             (Ok(read), buf)
169 | |         }
    | |_________^ the trait `IoVecBufMut` is not implemented for `T`
    |
note: required by a bound in `<A as AsyncReadRentExt>::read_vectored_exact`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_read_rent_ext.rs:144:22
    |
139 |     fn read_vectored_exact<T: 'static>(
    |        ------------------- required by a bound in this
...
144 |         T: 'static + IoVecBufMut,
    |                      ^^^^^^^^^^^ required by this bound in `<A as AsyncReadRentExt>::read_vectored_exact`
help: consider further restricting this bound
    |
137 |     type ReadVectoredExactFuture<'a, T> = impl Future<Output = BufResult<usize, T>> where A: 'a, T: 'a + buf::io_vec_buf::IoVecBufMut;
    |                                                                                                        ++++++++++++++++++++++++++++++

error[E0277]: the trait bound `T: IoBuf` is not satisfied
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_write_rent_ext.rs:43:9
   |
43 | /         async move {
44 | |             let ptr = buf.read_ptr();
45 | |             let len = buf.bytes_init();
46 | |             let mut written = 0;
...  |
64 | |             (Ok(written), buf)
65 | |         }
   | |_________^ the trait `IoBuf` is not implemented for `T`
   |
note: required by a bound in `<A as AsyncWriteRentExt>::write_all`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_write_rent_ext.rs:41:22
   |
39 |     fn write_all<T>(&mut self, buf: T) -> Self::WriteExactFuture<'_, T>
   |        --------- required by a bound in this
40 |     where
41 |         T: 'static + IoBuf,
   |                      ^^^^^ required by this bound in `<A as AsyncWriteRentExt>::write_all`
help: consider further restricting this bound
   |
37 |     type WriteExactFuture<'a, T> = impl Future<Output = BufResult<usize, T>> where A: 'a, T: 'a + buf::io_buf::IoBuf;
   |                                                                                                 ++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBuf` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_write_rent_ext.rs:78:9
    |
78  | /         async move {
79  | |             while written < len {
80  | |                 let (res, meta_) = self.writev(meta).await;
81  | |                 meta = meta_;
...   |
100 | |             (Ok(written), buf)
101 | |         }
    | |_________^ the trait `IoVecBuf` is not implemented for `T`
    |
note: required by a bound in `<A as AsyncWriteRentExt>::write_vectored_all`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/async_write_rent_ext.rs:72:22
    |
70  |     fn write_vectored_all<T>(&mut self, buf: T) -> Self::WriteVectoredExactFuture<'_, T>
    |        ------------------ required by a bound in this
71  |     where
72  |         T: 'static + IoVecBuf,
    |                      ^^^^^^^^ required by this bound in `<A as AsyncWriteRentExt>::write_vectored_all`
help: consider further restricting this bound
    |
68  |     type WriteVectoredExactFuture<'a, T> = impl Future<Output = BufResult<usize, T>> where A: 'a, T: 'a + buf::io_vec_buf::IoVecBuf;
    |                                                                                                         +++++++++++++++++++++++++++

error[E0277]: expected a `FnMut<(<T as io::stream::Stream>::Item,)>` closure, found `F`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/stream/stream_ext.rs:34:9
   |
34 | /         async move {
35 | |             while let Some(item) = self.next().await {
36 | |                 (f)(item).await;
37 | |             }
38 | |         }
   | |_________^ expected an `FnMut<(<T as io::stream::Stream>::Item,)>` closure, found `F`
   |
note: required by a bound in `stream_ext::StreamExt::for_each`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/stream/stream_ext.rs:30:12
   |
28 |     fn for_each<Fut, F>(mut self, mut f: F) -> ForEachFut<Self, Fut, F>
   |        -------- required by a bound in this
29 |     where
30 |         F: FnMut(Self::Item) -> Fut,
   |            ^^^^^^^^^^^^^^^^^^^^^^^^ required by this bound in `stream_ext::StreamExt::for_each`
help: consider restricting type parameter `F`
   |
44 | type ForEachFut<T: Stream, Fut, F: std::ops::FnMut<(<T as io::stream::Stream>::Item,)>> = impl Future<Output = ()>;
   |                                  +++++++++++++++++++++++++++++++++++++++++++++++++++++

error[E0277]: `Fut` is not a future
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/stream/stream_ext.rs:34:9
   |
34 | /         async move {
35 | |             while let Some(item) = self.next().await {
36 | |                 (f)(item).await;
37 | |             }
38 | |         }
   | |_________^ `Fut` is not a future
   |
   = note: Fut must be a future or must implement `IntoFuture` to be awaited
note: required by a bound in `stream_ext::StreamExt::for_each`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/stream/stream_ext.rs:31:14
   |
28 |     fn for_each<Fut, F>(mut self, mut f: F) -> ForEachFut<Self, Fut, F>
   |        -------- required by a bound in this
...
31 |         Fut: Future<Output = ()>,
   |              ^^^^^^^^^^^^^^^^^^^ required by this bound in `stream_ext::StreamExt::for_each`
help: consider restricting type parameter `Fut`
   |
44 | type ForEachFut<T: Stream, Fut: futures::Future, F> = impl Future<Output = ()>;
   |                               +++++++++++++++++

error[E0277]: the trait bound `T: IoBufMut` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_reader.rs:78:9
    |
78  | /         async move {
79  | |             // If we don't have any buffered data and we're doing a massive read
80  | |             // (larger than our internal buffer), bypass our internal buffer
81  | |             // entirely.
...   |
100 | |             (Ok(amt), buf)
101 | |         }
    | |_________^ the trait `IoBufMut` is not implemented for `T`
    |
note: required by a bound in `<buf_reader::BufReader<R> as AsyncReadRent>::read`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_reader.rs:77:16
    |
77  |     fn read<T: crate::buf::IoBufMut>(&mut self, mut buf: T) -> Self::ReadFuture<'_, T> {
    |                ^^^^^^^^^^^^^^^^^^^^ required by this bound in `<buf_reader::BufReader<R> as AsyncReadRent>::read`
help: consider further restricting this bound
    |
72  |         T: 'a + buf::io_buf::IoBufMut, R: 'a;
    |               +++++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBufMut` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_reader.rs:105:9
    |
105 | /         async move {
106 | |             let n = match unsafe { RawBuf::new_from_iovec_mut(&mut buf) } {
107 | |                 Some(raw_buf) => self.read(raw_buf).await.0,
108 | |                 None => Ok(0),
...   |
113 | |             (n, buf)
114 | |         }
    | |_________^ the trait `IoVecBufMut` is not implemented for `T`
    |
note: required by a bound in `<buf_reader::BufReader<R> as AsyncReadRent>::readv`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_reader.rs:104:17
    |
104 |     fn readv<T: crate::buf::IoVecBufMut>(&mut self, mut buf: T) -> Self::ReadvFuture<'_, T> {
    |                 ^^^^^^^^^^^^^^^^^^^^^^^ required by this bound in `<buf_reader::BufReader<R> as AsyncReadRent>::readv`
help: consider further restricting this bound
    |
75  |         T: 'a + buf::io_vec_buf::IoVecBufMut, R: 'a;
    |               ++++++++++++++++++++++++++++++

error[E0277]: the trait bound `T: IoBuf` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_writer.rs:100:9
    |
100 | /         async move {
101 | |             let owned_buf = self.buf.as_ref().unwrap();
102 | |             let owned_len = owned_buf.len();
103 | |             let amt = buf.bytes_init();
...   |
134 | |             }
135 | |         }
    | |_________^ the trait `IoBuf` is not implemented for `T`
    |
note: required by a bound in `<buf_writer::BufWriter<W> as AsyncWriteRent>::write`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_writer.rs:99:17
    |
99  |     fn write<T: crate::buf::IoBuf>(&mut self, buf: T) -> Self::WriteFuture<'_, T> {
    |                 ^^^^^^^^^^^^^^^^^ required by this bound in `<buf_writer::BufWriter<W> as AsyncWriteRent>::write`
help: consider further restricting this bound
    |
88  |         T: 'a + buf::io_buf::IoBuf, W: 'a;
    |               ++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBuf` is not satisfied
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_writer.rs:140:9
    |
140 | /         async move {
141 | |             let n = match unsafe { RawBuf::new_from_iovec(&buf_vec) } {
142 | |                 Some(raw_buf) => self.write(raw_buf).await.0,
143 | |                 None => Ok(0),
144 | |             };
145 | |             (n, buf_vec)
146 | |         }
    | |_________^ the trait `IoVecBuf` is not implemented for `T`
    |
note: required by a bound in `<buf_writer::BufWriter<W> as AsyncWriteRent>::writev`
   --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/buf_writer.rs:139:18
    |
139 |     fn writev<T: crate::buf::IoVecBuf>(&mut self, buf_vec: T) -> Self::WritevFuture<'_, T> {
    |                  ^^^^^^^^^^^^^^^^^^^^ required by this bound in `<buf_writer::BufWriter<W> as AsyncWriteRent>::writev`
help: consider further restricting this bound
    |
91  |         T: 'a + buf::io_vec_buf::IoVecBuf, W: 'a;
    |               +++++++++++++++++++++++++++

error[E0277]: the trait bound `T: IoBufMut` is not satisfied
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/prefixed_io.rs:45:9
   |
45 | /         async move {
46 | |             if buf.bytes_total() == 0 {
47 | |                 return (Ok(0), buf);
48 | |             }
...  |
68 | |             self.io.read(buf).await
69 | |         }
   | |_________^ the trait `IoBufMut` is not implemented for `T`
   |
note: required by a bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::read`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/prefixed_io.rs:44:16
   |
44 |     fn read<T: crate::buf::IoBufMut>(&mut self, mut buf: T) -> Self::ReadFuture<'_, T> {
   |                ^^^^^^^^^^^^^^^^^^^^ required by this bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::read`
help: consider further restricting this bound
   |
38 |         T: 'a + buf::io_buf::IoBufMut, Self: 'a;
   |               +++++++++++++++++++++++

error[E0277]: the trait bound `T: IoVecBufMut` is not satisfied
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/prefixed_io.rs:73:9
   |
73 | /         async move {
74 | |             let n = match unsafe { RawBuf::new_from_iovec_mut(&mut buf) } {
75 | |                 Some(raw_buf) => self.read(raw_buf).await.0,
76 | |                 None => Ok(0),
...  |
81 | |             (n, buf)
82 | |         }
   | |_________^ the trait `IoVecBufMut` is not implemented for `T`
   |
note: required by a bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::readv`
  --> /root/.cargo/registry/src/mirrors.ustc.edu.cn-12df342d903acd47/monoio-0.0.6/src/io/util/prefixed_io.rs:72:17
   |
72 |     fn readv<T: crate::buf::IoVecBufMut>(&mut self, mut buf: T) -> Self::ReadvFuture<'_, T> {
   |                 ^^^^^^^^^^^^^^^^^^^^^^^ required by this bound in `<PrefixedReadIo<I, P> as AsyncReadRent>::readv`
help: consider further restricting this bound
   |
42 |         T: 'a + buf::io_vec_buf::IoVecBufMut, Self: 'a;
   |               ++++++++++++++++++++++++++++++

For more information about this error, try `rustc --explain E0277`.
error: could not compile `monoio` due to 14 previous errors

Start error using s2n_quic

I am using monoio with s2n_quic and when i wanted to start the server this error showed up:

Error: StartError(there is no reactor running, must be called from the context of a Tokio 1.x runtime)

My code:

use s2n_quic::Server;
use std::error::Error;
use monoio;

#[monoio::main]
async fn main() -> Result<(), Box<dyn Error>> {
    println!("start ecexc");
    let mut server = Server::builder()
        .with_io("127.0.0.1:4433")?
        .start()?;
    println!("end");

    while let Some(mut connection) = server.accept().await {
      
        monoio::spawn(async move {
            while let Ok(Some(mut stream)) = connection.accept_bidirectional_stream().await {
               
                monoio::spawn(async move {
                    
                    while let Ok(Some(data)) = stream.receive().await {
                        stream.send(data).await.expect("stream should be open");
                    }
                });
            }
        });
    }

    Ok(())
}

cargo test failed

Version
0.0.3

cargo tree | grep monoio

monoio v0.0.3 (/home/sebo/Source/github/monoio/monoio) ├── monoio-macros v0.0.2 (proc-macro) (/home/sebo/Source/github/monoio/monoio-macros) monoio-compat v0.0.4 (/home/sebo/Source/github/monoio/monoio-compat) ├── monoio v0.0.3 (/home/sebo/Source/github/monoio/monoio) (*) └── monoio v0.0.3 (/home/sebo/Source/github/monoio/monoio) (*) monoio-examples v0.0.0 (/home/sebo/Source/github/monoio/examples) ├── monoio v0.0.3 (/home/sebo/Source/github/monoio/monoio) (*) ├── monoio-compat v0.0.4 (/home/sebo/Source/github/monoio/monoio-compat) (*)

Platform
The output of uname -a and ulimit -l.
➜ monoio git:(master) uname -a [10/12/21 | 1:03:18] Linux sebo-nuc10i7fnh 5.10.79-1-MANJARO #1 SMP PREEMPT Fri Nov 12 20:26:09 UTC 2021 x86_64 GNU/Linux ➜ monoio git:(master) ulimit -l [10/12/21 | 1:03:20] 64

Description
cargo test failed

`running 1 test
test tcp::tests::test_rw ... FAILED

failures:

---- tcp::tests::test_rw stdout ----
thread 'tcp::tests::test_rw' panicked at 'Failed building the Runtime: Os { code: 12, kind: OutOfMemory, message: "Cannot allocate memory" }', monoio-compat/src/tcp.rs:143:38
note: run with RUST_BACKTRACE=1 environment variable to display a backtrace

failures:
tcp::tests::test_rw

test result: FAILED. 0 passed; 1 failed; 0 ignored; 0 measured; 0 filtered out; finished in 0.00s`

gRPC example with Tonic

Would like to see gRPC example by implementing traits from Tonic.

Happy to PR something in if successful.

OpenOptions allow custom flags

Is your feature request related to a problem? Please describe.
monoio::fs::OpenOptions at present only permits specifying a limited number of options. Some use cases may prefer O_DIRECT options when open() an fd.

tokio-uring has its OpenOptions implement OpenOptionsExt trait. I guess monoio should have something similar.

Describe the solution you'd like
Make monoio work with direct IO if developers prefer.

Describe alternatives you've considered
tokio-uring

Additional context
N/A

proposal: Make monoio usable on stable rust

Is your feature request related to a problem? Please describe.

opendal intends to empower everyone to painlessly and efficiently access different storage services.

We have stargle a lot with tokio-fs's poor performance for our fs service, so we are evaluating the use of monoio as our fs backend. We only want to use local file IO (read, write, seek, statx and so no). No networking, no socket, no stream, no sink...

However, monoio is nightly only and can't meet opendal build requirements. So I'm thinking can we make monoio usable on stable rust?

monoio is using the following unstable features:

#![feature(generic_associated_types)]
#![feature(type_alias_impl_trait)]
#![feature(box_into_inner)]
#![feature(new_uninit)]
#![feature(io_error_more)]
#![feature(stmt_expr_attributes)]
#![feature(unboxed_closures)]
#![feature(once_cell)]

With hiding net, io under a specified feature, we have no depends on GAT, and monoio can be built on stable rust!

Describe the solution you'd like

Hiding net, io from public API with the specified feature so that monoio is usable on stable rust. And users can enable them with features like nightly if they do want to use monoio on nightly rust.

Describe alternatives you've considered

Add a new feature to hide net and io.

This way can prevent breaking the existing build.

Additional context

Apart from generic_associated_types and type_alias_impl_trait, we should also rewrite some of the code to remove the dependencies on features like box_into_inner. Or hide under feature too.

A possible demo: #96

Fixed buffers to support ReadFixed and WriteFixed ops

Provides fixed buffers from a pre-registered buffer pool, to support ReadFixed and WriteFixed ops. The APIs monoio should provide are important and should be discussed.

There are several related works. glommio uses a per-reactor buddy-system allocator to allocate buffers for writes to their DmaFile. The DmaFile needs the buffer and the position be properly aligned for Direct I/O. But monoio doesn't suppor Direct I/O now. In the tokio-uring's pending PR, the APIs are as follows:

struct FixedBuf {}
struct BufRegistry {}

impl BufRegistry {
    pub fn new(bufs: impl IntoIterator<Item = Vec<u8>>) -> Self {}
    pub fn register(&self) -> io::Result<()> {}
    pub fn unregister(&self) -> io::Result<()> {}
    pub fn check_out(&self, index: usize) -> Option<FixedBuf> {}
}

impl File {
    pub async fn read_fixed_at(
        &self,
        buf: Slice<FixedBuf>,
        pos: u64,
    ) -> crate::BufResult<usize, Slice<FixedBuf>> {}

    pub async fn write_fixed_at(
        &self,
        buf: Slice<FixedBuf>,
        pos: u64,
    ) -> crate::BufResult<usize, Slice<FixedBuf>> {}
}

Should monoio provide extra read_fixed_at and write_fixed_at methods which only accept FixedBuf as parameter, or extend IoBuf/IoBufMut to provide a is_registered/is_fixed hint to allow calling Op<ReadFixed/WriteFixed> internally in read_at/write_at?

How the fixed buffer allocator/registry should be like? I prefer to provide a allocator like glommio, and provide a register buffers method, but let users manage these fixed buffers, monoio only take and return their ownerships.

I'm glad to implement this feature as long as the discussion ends up with a result.

[GLCC 赛题] 使用纯异步 Rust 实现高性能可扩展的通用网关系统

🚩GLCC 高校编程夏令营赛题
👏欢迎大家参与~

GLCC 是由**计算机学会举办的高校开源编程夏令营活动,活动为项目准备了开源奖学金。请在以下链接中报名参与⬇️
报名链接: https://www.gitlink.org.cn/glcc/subjects/detail/287
关于issue相关问题、解题思路的答疑将在群内进行

image

1、题目描述
目前生产环境中大量 Nginx 作为入口网关。而由于 Nginx 使用 C 语言编写,且高度耦合底层系统调用,所以在可扩展性上较为欠缺,二次开发成本较高,且组件不易复用。多年来尽管拥有能力极强的贡献者,还是产生了很多由于内存安全导致的漏洞,影响生产环境安全性。
近年来 Caddy 补足了 Nginx 的上述问题,并集成 ACME 大大提高了易用性,但是使用 Golang 开发在降低二次开发难度的同时也带来了一系列的性能问题(有 Go 编译器本身的原因,也有 GC 和网络模型的原因,会导致在大规格机器上表现较差),不具备大规模生产可用性。
Rust 语言兼具了性能和内存安全,对于 Gateway 是最理想的语言。基于我们开源的 Monoio,可以在不牺牲性能的前提下,做到对平台(Linux、Windows 等)、底层系统调用(如 io_uring 与 epoll)的屏蔽。
使用 Rust 开发纯异步的通用网关系统,可以为生产环境提供可证明的安全性、最佳的性能和可扩展性。

2、编码任务

  • 基于 Monoio 完善 HTTP 框架
  • 实现可插拔 Gateway 主框架,并实现 TCP / h1 / h2 组件
  • ACME 支持(类似 Caddy)
  • QUIC 支持

3、技能要求和编程语言
Rust 语言、网络编程

4、预期完成结果

  • 提供可扩展的生产可用的 Gateway
  • 较完善的 TCP / h1 / h2 / QUIC / ACME 组件支持

Stable Rust now that GATs are stable?

Thanks for making this library!

I noticed in the readme that GATs are a reason for depending on nightly Rust instead of stable. Now that GATs have just landed in stable, is it possible for monoio to depend on stable?

IoUring setup options

Is your feature request related to a problem? Please describe.
crate io-uring exposes several setup options, allowing advanced use cases and features described in section 8 of Efficient IO with io_uring. Features like Polled IO, Fixed IO buffers, and kernel side polling are helpful to applications with high IOPS and latency sensitivity.

Describe the solution you'd like
RuntimeBuilder accepts pre-built IoUring

Describe alternatives you've considered
N/A

Additional context
N/A

TcpListener/ListenerConfig support configure IPV6_V6ONLY

Is your feature request related to a problem? Please describe.
I want TcpListener bind both ipv4 and ipv6 (dual stack)

Describe the solution you'd like
Add a config in struct ListenerConfig for ipv6_v6only

Describe alternatives you've considered
None

Additional context

[w@ww ~]$ sysctl net.ipv6.bindv6only
net.ipv6.bindv6only = 0

image

High CPU usage

Version
v0.0.3

Platform
Linux linux 5.11.0-40-generic #44~20.04.2-Ubuntu SMP Tue Oct 26 18:07:44 UTC 2021 x86_64 x86_64 x86_64 GNU/Linux

ulimit -l 65536

Description

All examples eat up ~100% of CPU

cargo run --color=always --example echo --release or
cargo run --color=always --example proxy --release or
cargo run --color=always --example hyper-server --release

Screenshot from 2021-12-15 23-38-31

TcpStream::connect fails with InvalidInput while running tcp_uring example

Version
I use current master at 2f151f2

Platform
The output of uname -a and ulimit -l.

uname -a
Linux 5.17.14-300.fc36.x86_64+debug #1 SMP PREEMPT Thu Jun 9 13:21:53 UTC 2022 x86_64 x86_64 x86_64 GNU/Linux

ulimit -l
8192

Description

TcpStream::connect fails with InvalidInput while running tcp_uring example

Running cargo run --example tcp_uring yields

Will run with IoUringDriver(you must be on linux and enable iouring feature)
[Client] Waiting for server ready
[Server] Bind ready
[Client] Server is ready, will connect and send data
thread '<unnamed>' panicked at '[Client] Unable to connect to server: Os { code: 22, kind: InvalidInput, message: "Invalid argument" }', examples/tcp_uring.rs:34:18
Full backrace
stack backtrace:
 0:     0x5613d465c810 - std::backtrace_rs::backtrace::libunwind::trace::h60bf900414c5dcf2
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/../../backtrace/src/backtrace/mod.rs:66:5
 1:     0x5613d465c810 - std::backtrace_rs::backtrace::trace_unsynchronized::habfc3ef92583c158
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/../../backtrace/src/backtrace/mod.rs:66:5
 2:     0x5613d465c810 - std::sys_common::backtrace::_print_fmt::hae196fe4153d0b63
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:66:5
 3:     0x5613d465c810 - <std::sys_common::backtrace::_print::DisplayBacktrace as core::fmt::Display>::fmt::h08f9c50f850514f5
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:45:22
 4:     0x5613d4678a0c - core::fmt::write::hcd15d2c673b5a9c1
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/fmt/mod.rs:1198:17
 5:     0x5613d4658ed5 - std::io::Write::write_fmt::h417be4d25b915a22
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/io/mod.rs:1672:15
 6:     0x5613d465e071 - std::sys_common::backtrace::_print::h95ab226f37d571aa
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:48:5
 7:     0x5613d465e071 - std::sys_common::backtrace::print::hc77fb9ed6b2eeb3d
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:35:9
 8:     0x5613d465e071 - std::panicking::default_hook::{{closure}}::hca418bd67097eb22
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:295:22
 9:     0x5613d465dd3e - std::panicking::default_hook::h1df4f10c6ed28c53
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:314:9
10:     0x5613d465e713 - std::panicking::rust_panic_with_hook::hf1e7a1cb721a9823
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:698:17
11:     0x5613d465e607 - std::panicking::begin_panic_handler::{{closure}}::h573a00e95ce9340d
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:588:13
12:     0x5613d465cd34 - std::sys_common::backtrace::__rust_end_short_backtrace::h597b7803117e52fd
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:138:18
13:     0x5613d465e332 - rust_begin_unwind
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:584:5
14:     0x5613d45d49b3 - core::panicking::panic_fmt::h62ccf03c8a8a51b5
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/panicking.rs:142:14
15:     0x5613d45d4a43 - core::result::unwrap_failed::hff48f82f03d418ae
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/result.rs:1814:5
16:     0x5613d45e4eec - core::result::Result<T,E>::expect::h06b932c75fc0619f
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/result.rs:1064:23
17:     0x5613d45ed061 - tcp_uring::run::{{closure}}::{{closure}}::h798ebb573da54447
                             at /home/dr/checkout/monoio/examples/tcp_uring.rs:32:28
18:     0x5613d45e2dfc - <core::future::from_generator::GenFuture<T> as core::future::future::Future>::poll::h1d329dafc8cb7ef7
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/future/mod.rs:91:19
19:     0x5613d45e4596 - monoio::task::core::CoreStage<T>::poll::{{closure}}::hdf835fe76cf14c85
                             at /home/dr/checkout/monoio/monoio/src/task/core.rs:116:17
20:     0x5613d45d8480 - <core::cell::UnsafeCell<T> as monoio::task::utils::UnsafeCellExt<T>>::with_mut::h7635711df777ce23
                             at /home/dr/checkout/monoio/monoio/src/task/utils.rs:14:9
21:     0x5613d45e4663 - monoio::task::core::CoreStage<T>::with_mut::h56a469e0a03b69cb
                             at /home/dr/checkout/monoio/monoio/src/task/core.rs:101:9
22:     0x5613d45e4277 - monoio::task::core::CoreStage<T>::poll::h5af03ee54b8b3bb1
                             at /home/dr/checkout/monoio/monoio/src/task/core.rs:106:13
23:     0x5613d45df5f8 - monoio::task::harness::poll_future::hf77210d8be4f08a2
                             at /home/dr/checkout/monoio/monoio/src/task/harness.rs:376:18
24:     0x5613d45df6c7 - monoio::task::harness::Harness<T,S>::poll_inner::h9b1c2d534ff09919
                             at /home/dr/checkout/monoio/monoio/src/task/harness.rs:76:19
25:     0x5613d45e0973 - monoio::task::harness::Harness<T,S>::poll::h221813ad72ba1651
                             at /home/dr/checkout/monoio/monoio/src/task/harness.rs:52:15
26:     0x5613d45daa30 - monoio::task::raw::poll::hcf302bf61c74505b
                             at /home/dr/checkout/monoio/monoio/src/task/raw.rs:105:5
27:     0x5613d46172af - monoio::task::raw::RawTask::poll::h6674ecdee16f6556
                             at /home/dr/checkout/monoio/monoio/src/task/raw.rs:80:18
28:     0x5613d45f4365 - monoio::task::Task<S>::run::h85fa75e0221e7f99
                             at /home/dr/checkout/monoio/monoio/src/task/mod.rs:47:9
29:     0x5613d45e7ff3 - monoio::runtime::Runtime<D>::block_on::{{closure}}::{{closure}}::he6183734a8543936
                             at /home/dr/checkout/monoio/monoio/src/runtime.rs:143:29
30:     0x5613d45d4e63 - scoped_tls::ScopedKey<T>::set::ha00816e2f74cf61c
                             at /home/dr/.cargo/registry/src/github.com-1ecc6299db9ec823/scoped-tls-1.0.0/src/lib.rs:137:9
31:     0x5613d45e7a22 - monoio::runtime::Runtime<D>::block_on::{{closure}}::h5d57811c8d0fbcc9
                             at /home/dr/checkout/monoio/monoio/src/runtime.rs:130:13
32:     0x5613d45d50c3 - scoped_tls::ScopedKey<T>::set::hdc18c92f0ae942b1
                             at /home/dr/.cargo/registry/src/github.com-1ecc6299db9ec823/scoped-tls-1.0.0/src/lib.rs:137:9
33:     0x5613d45ea1db - <monoio::driver::uring::IoUringDriver as monoio::driver::Driver>::with::hd2978ab165e731a3
                             at /home/dr/checkout/monoio/monoio/src/driver/uring/mod.rs:259:9
34:     0x5613d45e7790 - monoio::runtime::Runtime<D>::block_on::h0470bf77d9dca20f
                             at /home/dr/checkout/monoio/monoio/src/runtime.rs:129:9
35:     0x5613d45df293 - monoio::start::h8fbcb4bfaa65ffce
                             at /home/dr/checkout/monoio/monoio/src/lib.rs:94:5
36:     0x5613d45ecbe7 - tcp_uring::run::{{closure}}::h34b374f24a920d28
                             at /home/dr/checkout/monoio/examples/tcp_uring.rs:27:9
37:     0x5613d45f371e - std::sys_common::backtrace::__rust_begin_short_backtrace::h7a904a254d5f590b
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys_common/backtrace.rs:122:18
38:     0x5613d45dcd9a - std::thread::Builder::spawn_unchecked_::{{closure}}::{{closure}}::hd149554c9dcdfa01
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/thread/mod.rs:505:17
39:     0x5613d45e0f7e - <core::panic::unwind_safe::AssertUnwindSafe<F> as core::ops::function::FnOnce<()>>::call_once::h3815bd578e1d88bc
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/panic/unwind_safe.rs:271:9
40:     0x5613d45d8da3 - std::panicking::try::do_call::h6df9a0604e18405a
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:492:40
41:     0x5613d45d926b - __rust_try
42:     0x5613d45d898a - std::panicking::try::h7d27a3888d9bc7c1
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panicking.rs:456:19
43:     0x5613d45e498e - std::panic::catch_unwind::h9eb5f3a8371e0827
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/panic.rs:137:14
44:     0x5613d45dcbe5 - std::thread::Builder::spawn_unchecked_::{{closure}}::h8819e20da09f5953
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/thread/mod.rs:504:30
45:     0x5613d45efb2e - core::ops::function::FnOnce::call_once{{vtable.shim}}::h947bc7d58899b370
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/core/src/ops/function.rs:248:5
46:     0x5613d4660b13 - <alloc::boxed::Box<F,A> as core::ops::function::FnOnce<Args>>::call_once::hede4885ee64a1fac
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/alloc/src/boxed.rs:1935:9
47:     0x5613d4660b13 - <alloc::boxed::Box<F,A> as core::ops::function::FnOnce<Args>>::call_once::h06c745de9f474954
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/alloc/src/boxed.rs:1935:9
48:     0x5613d4660b13 - std::sys::unix::thread::Thread::new::thread_start::h529a493f2fb191b6
                             at /rustc/d394408fb38c4de61f765a3ed5189d2731a1da91/library/std/src/sys/unix/thread.rs:108:17
49:     0x7fb81032ce2d - start_thread
50:     0x7fb8103b21b0 - __clone3
51:                0x0 - <unknown>

Let me know if I can add more details. Thanks!

Cannot compile on rustc nightly 1.60

rustc --version
rustc 1.60.0-nightly (bfe156467 2022-01-22)
error[E0271]: type mismatch resolving `<impl futures::Future<Output = [async output]> as futures::Future>::Output == ()`
  --> /home/jack/.cargo/registry/src/github.com-1ecc6299db9ec823/monoio-0.0.3/src/stream/stream_ext.rs:44:30
   |
34 |           async move {
   |  ____________________-
35 | |             while let Some(item) = self.next().await {
36 | |                 (f)(item);
37 | |             }
38 | |         }
   | |_________- the found `async` block
...
44 |   type ForEachFut<T, Fut, F> = impl Future<Output = ()>;
   |                                ^^^^^^^^^^^^^^^^^^^^^^^^ expected `()`, found associated type
   |
  ::: /home/jack/.rustup/toolchains/nightly-x86_64-unknown-linux-gnu/lib/rustlib/src/rust/library/core/src/future/mod.rs:65:43
   |
65 |   pub const fn from_generator<T>(gen: T) -> impl Future<Output = T::Return>
   |                                             ------------------------------- the found opaque type
   |
   = note:    expected unit type `()`
           found associated type `<impl futures::Future<Output = [async output]> as futures::Future>::Output`
   = help: consider constraining the associated type `<impl futures::Future<Output = [async output]> as futures::Future>::Output` to `()`
   = note: for more information, visit https://doc.rust-lang.org/book/ch19-03-advanced-traits.html


error[E0277]: the trait bound `T: stream::Stream` is not satisfied
  --> /home/jack/.cargo/registry/src/github.com-1ecc6299db9ec823/monoio-0.0.3/src/stream/stream_ext.rs:44:30
   |
44 | type ForEachFut<T, Fut, F> = impl Future<Output = ()>;
   |                              ^^^^^^^^^^^^^^^^^^^^^^^^ the trait `stream::Stream` is not implemented for `T`
   |
help: consider restricting type parameter `T`
   |
44 | type ForEachFut<T: stream::Stream, Fut, F> = impl Future<Output = ()>;
   |                  ++++++++++++++++



[Draft] Scoped Future

Motivation

In current, monoio::spawn requires Future and its output must be 'static. However, user maybe takes lots of needless overhead (such as Rc to those captured objects) to promise 'static. If we consider the Future: ?Send + ?Sync requirment of monoio::spawn, removing the 'static requirement should be possible, because the lifetime of JoinHandle is able to be explicit and it can not be shared between threads. It is something like the future version of scoped-threads.

Advantages

Scoped-future allows user use reference (&'future _ / &'future mut _) rather than Rc or Rc<RefCell<_>> to those object which are captured in async block, it is useful to reduce the runtime overhead in some perfomance-sensitive cases.

Possible Design

Add scoped_spawn function and it returns ScopedJoinHandle:

pub fn scoped_spawn<'future, T>(future: T) -> ScopedJoinHandle<'future, T::Output>
where
    T: Future + 'future,
    T::Output: 'future,

ScopedJoinHandle must be awaited or setted canceled in lifetime. In first case, maybe we are able to use #[must_use] lint to suggest users, and force deallocate the raw task in <ScopedJoinHandle as Drop>::drop in second case to prevent data race.

Wrong version for monoio-macro on Crates.io

Version

monoio = "0.0.3"

Description

The following code failed to compile:

#[monoio::main(worker_threads = 2)]
async fn main() {
   println!("123");
}

with error: error: Unknown attribute worker_threads is specified; expected one of: worker_threads, entries, timer_enabled

Monoio-macro on Crates.io: https://docs.rs/monoio-macros/0.0.2/src/monoio_macros/entry.rs.html#110

match ident.as_str() {
                    "entries" => {
                        config.set_entries(
                            namevalue.lit.clone(),
                            syn::spanned::Spanned::span(&namevalue.lit),
                        )?;
                    }
                    "timer_enabled" | "timer" => config.set_timer_enabled(
                        namevalue.lit.clone(),
                        syn::spanned::Spanned::span(&namevalue.lit),
                    )?,
                    name => {
                        let msg = format!(
                            "Unknown attribute {} is specified; expected one of: `worker_threads`, `entries`, `timer_enabled`",
                            name,
                        );
                        return Err(syn::Error::new_spanned(namevalue, msg));
                    }
                }

On GitHub:

match ident.as_str() {
                    "entries" => config.set_entries(
                        namevalue.lit.clone(),
                        syn::spanned::Spanned::span(&namevalue.lit),
                    )?,
                    "timer_enabled" | "enable_timer" | "timer" => config.set_timer_enabled(
                        namevalue.lit.clone(),
                        syn::spanned::Spanned::span(&namevalue.lit),
                    )?,
                    "worker_threads" | "workers" | "threads" => config.set_threads(
                        namevalue.lit.clone(),
                        syn::spanned::Spanned::span(&namevalue.lit),
                    )?,
                    name => {
                        let msg = format!(
                            "Unknown attribute {} is specified; expected one of: `worker_threads`, `entries`, `timer_enabled`",
                            name,
                        );
                        return Err(syn::Error::new_spanned(namevalue, msg));
                    }
                }

Does code on Crates.io have missed something?

need support kernel multishot operation.

io_uring has added operations such as IORING_OP_ACCEPT and IORING_OP_POLL_ADD in the kernel update, but there is no related circuit diagram to support this kind of wiping.

Issue with RawWakerVTable on crossing threads

As a runtime of thread-per-core model, mostly we works like running multiple independent threads in parallel. So we implemented it in a non-thread-safe way. For example, when maintaining refcount of task, we use UnsafeCell<usize>.
There are 2 problems here:

  1. There is no non-Send + non-Sync version of Waker / RawWakerVTable. As the document of RawWakerVTable, all 4 functions must be thread safe.
    Since Waker and RawWakerVTable are all defined by std library and they impl Send and Sync, so we cannot restrict users not pass them between threads. Even we can tell users not pass them, this is not a rusty way. We'd better let it crash on compiling or running, or, we have to make it really thread-safe.
  2. We support a sync feature which implemented by passing waker that not belongs to current thread. This solves task execution problem on thread-per-core way, but the implementation itself is not thread safe. We have to transfrom the state in atomic way. The problems comes from the task module too, the same as the first problem. I believe problem #117 is triggered by this problem.

So as a user, if you do not await across threads, you are not affected by the issue. The issue will be fixed soon.

Hyper client example

Is your feature request related to a problem? Please describe.
Users need to write a http client with monoio.

Describe the solution you'd like
Implement a http client using hyper based on monoio-compat.

Describe alternatives you've considered
Maybe fork hyper? We are indeed working on it. But to use hyper for now, we may use it based on monoio-compat.

Additional context

Working with Files?

I had a go at writing a basic program to copy the content of files. However, I found that I can't use monoio::io::copy or monoio::io::zero_copy, I understand because monoio::fs::File only implements positional I/O.

I found my way to BufReader which sounds like what I want, from the docs:

BufReader implements AsyncBufRead and AsyncReadRent, and if the inner io implements AsyncWriteRent, it will delegate the implementation.

However, I do not see an implementation unless the underlying AsyncReadRent.

I'd like to be able to copy or zero-copy file-based file descriptors in an obvious way using the functions provided by the library.

Is this possible currently, or is there some implementation missing?

create a VTable (raw_waker) in a thread 2 and call (wake_by_val) in thread 1

Clone https://github.com/botika/ntex and:

cd ntex-rt
RUST_BACKTRACE=1 cargo test --no-default-features --features=monoio --lib -- --test-threads=1 --nocapture builder::tests::test_async -- --nocapture

The trace:

  1: scoped_tls::ScopedKey<T>::with
             at /home//.cargo/registry/src/github.com-1ecc6299db9ec823/scoped-tls-1.0.0/src/lib.rs:168:9
   2: <monoio::scheduler::LocalScheduler as monoio::task::Schedule>::schedule
             at /home//.cargo/git/checkouts/monoio-91d866d0e012895e/023e657/monoio/src/scheduler.rs:9:9
   3: monoio::task::harness::Harness<T,S>::wake_by_val
             at /home//.cargo/git/checkouts/monoio-91d866d0e012895e/023e657/monoio/src/task/harness.rs:176:17
   4: monoio::task::waker::wake_by_val
             at /home//.cargo/git/checkouts/monoio-91d866d0e012895e/023e657/monoio/src/task/waker.rs:76:5
   5: core::task::wake::Waker::wake
             at /rustc/b8c35ca26b191bb9a9ac669a4b3f4d3d52d97fb1/library/core/src/task/wake.rs:276:18

According to my logs, wake_by_val is launched in a thread without Runtime running.
VTable is created on the new thread and called on the main thread. i.e. raw_waker is called in thread(2) and wake_by_val in thread(1), creating a single runtime in thread(2).
Sorry for this mess of issue. I'm short on time and I'm trying to get the case out without the ntex wrapper. But I thought you would like to know as soon as possible. I will try to debug the exact case and update the issue.

I have to disassemble the part of ntex but this way it is clearer.

        // std channel
        let (tx, rx) = mpsc::channel();
        // spawn read
        thread::spawn(move || {
            // create a ntex system with monoio feature
            let runner = crate::System::build().finish();
            // send ntex system ref
            tx.send(runner.system()).unwrap();
           // block until finish
            let _ = runner.run_until_stop();
        });
        // take system
        let sys = rx.recv().unwrap();
        // sleep 1 second ( If you delete this line it works )
        std::thread::sleep(std::time::Duration::from_secs(1));
        // and exec function in the system
        sys.arbiter().exec_fn(|| {});

Recommend Projects

  • React photo React

    A declarative, efficient, and flexible JavaScript library for building user interfaces.

  • Vue.js photo Vue.js

    🖖 Vue.js is a progressive, incrementally-adoptable JavaScript framework for building UI on the web.

  • Typescript photo Typescript

    TypeScript is a superset of JavaScript that compiles to clean JavaScript output.

  • TensorFlow photo TensorFlow

    An Open Source Machine Learning Framework for Everyone

  • Django photo Django

    The Web framework for perfectionists with deadlines.

  • D3 photo D3

    Bring data to life with SVG, Canvas and HTML. 📊📈🎉

Recommend Topics

  • javascript

    JavaScript (JS) is a lightweight interpreted programming language with first-class functions.

  • web

    Some thing interesting about web. New door for the world.

  • server

    A server is a program made to process requests and deliver data to clients.

  • Machine learning

    Machine learning is a way of modeling and interpreting data that allows a piece of software to respond intelligently.

  • Game

    Some thing interesting about game, make everyone happy.

Recommend Org

  • Facebook photo Facebook

    We are working to build community through open source technology. NB: members must have two-factor auth.

  • Microsoft photo Microsoft

    Open source projects and samples from Microsoft.

  • Google photo Google

    Google ❤️ Open Source for everyone.

  • D3 photo D3

    Data-Driven Documents codes.