Hi! Thanks for your insights!

@andrew the gist I sent
<https://gist.github.com/rdettai/d2f9bc59b31785c35dce792878976a19>does
start two runtimes and works. The constraint seems to be that you cannot
start a new runtime (or block on one) in the async thread pool, but it
seems legal to start a new one from the sync thread pool. But I'm not sure
you would benefit from doing that, so I guess there should be only one
global runtime.

@mahmut, your snippet references a lot of things from your private crate,
so it's kind of hard to understand what's happening exactly. I am unsure
why you think it would be good to make arrow and parquet async if "making a
database async is not a good thing". If async did decrease cache
efficiency, the datafusion part of the processing would not necessarily be
the one that would be suffering the most as batches will mostly be
processed in one go, without yielding. One might argue that you will have
less unwanted context switches as you can have less threads. But that is
all speculation, isn't it? It's very hard to know how cache will behave
without very specific benchmarks! I would already be extremely happy with a
threading model that is not too complex, never gets stuck and distributes
work pretty efficiently. What do you think?

Have a nice week!

Remi

Le ven. 13 nov. 2020 à 13:17, vertexclique vertexclique <
vertexcli...@gmail.com> a écrit :

> About the questions;
>
> - Does anybody know a way to make such a setup work?
>
> I have written thin wrapper using async_trait around the parquet reader in
> my current project, instead of I am writing it, if we have exposed that
> from parquet and push chunks via channels it would work at the consumer
> side asynchronously too.
>
> - Making Parquet async is extremely difficult and breaking, should we try
> > to do it [2] ?
>
> We should, with exporting everything async under mod asynch; and slowly
> move towards it. With the help of sync mpsc channels, it can be
> asynchronous.
>
> - Is the benefit of having DataFusion async really big? Should we maybe
> > have both a sync and an async API ?
>
>
> I am looking more from the parquet and arrow side. Making a database async
> is not a good thing from my point of view since it messes up caches. I
> prefer TpC architecture(with pinned threads of Bastion) which is what we
> are using internally. DataFusion can go in that direction instead of going
> fully async, you will see the performance improvement comes with it
> drastically. But this suggestion is TIOLI kind of...
>
> Best,
> Mahmut
>
> vertexclique vertexclique <vertexcli...@gmail.com>, 13 Kas 2020 Cum, 12:55
> tarihinde şunu yazdı:
>
> > Hi,
> >
> > Just a small contribution. We have made some adjustments at Signavio with
> > Jörn to make the chunk reading from S3 streaming at least. You can find
> the
> > code here. Mind that for this to make a bigger effect increase the
> hyper's
> > buffer size in its' configuration (which is not included in this code):
> > https://gist.github.com/vertexclique/b9bcfe5c1596d4d087e71b5d608c279b
> >
> > Best,
> > Mahmut
> >
> >
> > Andrew Lamb <al...@influxdata.com>, 13 Kas 2020 Cum, 12:35 tarihinde
> şunu
> > yazdı:
> >
> >> My understanding of tokio is that there is exactly one global Runtime
> >> <https://docs.rs/tokio/0.3.3/tokio/runtime/struct.Runtime.html> which
> has
> >> two thread pools: one for synchronous tasks and one for async tasks
> >>
> >> I am fairly sure there can be only one global Runtime (because when I
> >> tried
> >> try to explicitly create one when an existing one is present, tokio
> >> panic!'ed on me).
> >>
> >> The complexity created is definitely a concern. My personal feeling is
> >> that
> >> getting async pushed all the way down will be the least complex
> solution,
> >> though it still won't be "simple"
> >>
> >>
> >> On Fri, Nov 13, 2020 at 5:16 AM Rémi Dettai <rdet...@gmail.com> wrote:
> >>
> >> > Hi Andrew!
> >> >
> >> > Thanks for your quick response and sorry it took me so long to answer
> >> back.
> >> >
> >> > `spawn_blocking` solves the issue:
> >> > https://gist.github.com/rdettai/d2f9bc59b31785c35dce792878976a19
> >> >
> >> > I am still worried by the amount of thread pools and complexity it
> >> creates
> >> > (1 pool for the outer runtime, 1 pool for spawn_blocking, 1 pool for
> the
> >> > inner runtime). As you said, the best thing would be to push async all
> >> the
> >> > way down but it's pretty hard as it propagates through the entire
> >> codebase
> >> > :). For now I settled for adding async fetchers that download the data
> >> then
> >> > sync read from the in-memory buffers. I'll come back to this issue a
> bit
> >> > later because it still needs some adjustments.
> >> >
> >> > Remi
> >> >
> >> >
> >> > Le ven. 30 oct. 2020 à 11:27, Andrew Lamb <al...@influxdata.com> a
> >> écrit :
> >> >
> >> > > Tokio has a function `spawn_blocking`
> >> > > <https://docs.rs/tokio/0.3.2/tokio/task/fn.spawn_blocking.html>
> that
> >> > > allows
> >> > > running synchronous / blocking code as a future on the current
> >> runtime.
> >> > You
> >> > > can finagle pretty much any combination of sync / async using
> >> > > spawn_blocking and channels, though the resulting code may not be
> the
> >> > most
> >> > > beautiful.
> >> > >
> >> > > Once you introduce `async` into a project or use an `async` library
> >> like
> >> > > rusto, it feels to me like Rust leads you towards pushing async all
> >> the
> >> > way
> >> > > down and indeed the easiest thing for you, given your described
> >> > > usecase would be async all the way down.
> >> > >
> >> > > I personally think having an async implementation of parquet would
> be
> >> > very
> >> > > valuable, as more and more Rust uses tokio / async IO. Maybe we
> could
> >> > > implement an optional async interface on top of the blocking
> >> > > implementation.
> >> > >
> >> > > Likewise, having a sync api and an async api for DataFusion also
> seems
> >> > > valuable to to me.
> >> > >
> >> > > In my opinion, the biggest benefit from having DataFusion use
> >> tokio/async
> >> > > is a single unified thread pool and execution model for both CPU and
> >> IO
> >> > > work. Prior to being async-ized with the tokio thread pool,
> DataFusion
> >> > > spawned / managed threads on its own; Adding additional parallelism
> >> > without
> >> > > over subscribing the CPU was likely going to be a significant
> effort.
> >> > There
> >> > > is a thread
> >> > > <
> >> > >
> >> >
> >>
> https://lists.apache.org/thread.html/rbc4535613cb9af3467255234b49222bb8d3e57ef91790ebeff66aa74%40%3Cdev.arrow.apache.org%3E
> >> > > >
> >> > > on this mailing list about a similar challenge in the C++
> >> implementation,
> >> > > to give you a sense of the kinds of issues we are hoping to avoid in
> >> > > DataFusion with using async
> >> > >
> >> > > Andrew
> >> > >
> >> > >
> >> > > On Fri, Oct 30, 2020 at 4:28 AM Rémi Dettai <rdet...@gmail.com>
> >> wrote:
> >> > >
> >> > > > Hi everyone!
> >> > > >
> >> > > > If you are reading this, it means that you felt in the trap of my
> >> > catchy
> >> > > > (but meaningless) title!
> >> > > >
> >> > > > This discussion somewhat relates to [1].
> >> > > >
> >> > > > DataFusion has recently made its top level "actions" (collect,
> >> > write...)
> >> > > > async. The problem is that most of the codebase is not async (in
> >> > > particular
> >> > > > Parquet [2]), which means that you have to make an async context
> >> work
> >> > > > together with a sync one.
> >> > > >
> >> > > > This works okay... until it doesn't!
> >> > > >
> >> > > > I am trying to read into DataFusion from S3, using the AWS Rust
> SDK
> >> > > Rusoto.
> >> > > > The problem is that this SDK is itself async. This means that you
> >> end
> >> > up
> >> > > > with the following layers:
> >> > > > DataFusion (async) -> Parquet (sync) -> Rusoto (async)
> >> > > > As you might now, Tokio does not support blocking on a runtime
> from
> >> > > within
> >> > > > a runtime.
> >> > > >
> >> > > > This triggers a set of questions:
> >> > > > - Does anybody know a way to make such a setup work?
> >> > > > - Making Parquet async is extremely difficult and breaking, should
> >> we
> >> > try
> >> > > > to do it [2] ?
> >> > > > - Is the benefit of having DataFusion async really big? Should we
> >> maybe
> >> > > > have both a sync and an async API ?
> >> > > >
> >> > > > Thanks everybody and have a wonderful day.
> >> > > >
> >> > > > Regards,
> >> > > >
> >> > > > Remi
> >> > > >
> >> > > > [1] https://issues.apache.org/jira/browse/ARROW-9464
> >> > > > [2] https://issues.apache.org/jira/browse/ARROW-10307
> >> > > >
> >> > >
> >> >
> >>
> >
>

Reply via email to