Useful read on this topic today from the Julia language
https://julialang.org/blog/2019/07/multithreading
On Tue, Jul 23, 2019, 12:22 AM Jacques Nadeau wrote:
> There are two main things that have been important to us in Dremio around
> threading:
>
> Separate threading model from algorithms.
There are two main things that have been important to us in Dremio around
threading:
Separate threading model from algorithms. We chose to do parallelization at
the engine level instead of the operation level. This allows us to
substantially increase parallelization while still maintaining a
Le 22/07/2019 à 18:52, Wes McKinney a écrit :
>
> Probably the way is to introduce async-capable read APIs into the file
> interfaces. For example:
>
> file->ReadAsyncBlock(thread_ctx, ...);
>
> That way the file implementation can decide whether asynchronous logic
> is actually needed.
> I
On Mon, Jul 22, 2019 at 11:42 AM Antoine Pitrou wrote:
>
> On Mon, 22 Jul 2019 11:07:43 -0500
> Wes McKinney wrote:
> >
> > Right, which is why I'm suggesting a simple model to allow threads
> > that are waiting on IO to allow other threads to execute.
>
> If you are doing memory-mapped IO, how
On Mon, 22 Jul 2019 11:07:43 -0500
Wes McKinney wrote:
>
> Right, which is why I'm suggesting a simple model to allow threads
> that are waiting on IO to allow other threads to execute.
If you are doing memory-mapped IO, how do you plan to tell whether and
when you'll be going to wait for IO?
On Mon, Jul 22, 2019 at 10:49 AM Antoine Pitrou wrote:
>
>
> Le 18/07/2019 à 00:25, Wes McKinney a écrit :
> >
> > * We look forward in the stream until we find a complete Thrift data
> > page header. This may trigger 0 or more (possibly multiple) Read calls
> > to the underlying "file" handle.
Le 18/07/2019 à 00:25, Wes McKinney a écrit :
>
> * We look forward in the stream until we find a complete Thrift data
> page header. This may trigger 0 or more (possibly multiple) Read calls
> to the underlying "file" handle. In the default case, the data is all
> actually in memory so the
I've been looking at little bit at this in the context of Parquet files
One of the read hot paths in cpp/src/parquet is the function that
reads and decompresses data pages from the stream:
(SerializedPageReader::NextPage)
On Mon, 15 Jul 2019 11:49:56 -0500
Wes McKinney wrote:
>
> For example, suppose we had a thread pool with a limit of 8 concurrent
> tasks. Now 4 of them perform IO calls. Hypothetically this should
> happen:
>
> * Thread pool increments a "soft limit" to allow 4 more tasks to
> spawn, so at
On Mon, Jul 15, 2019 at 11:38 AM Antoine Pitrou wrote:
>
>
> Hi Anton,
>
> Le 12/07/2019 à 23:21, Malakhov, Anton a écrit :
> >
> > The result is that all these execution nodes scale well enough and run
> > under 100 milliseconds on my 2 x Xeon E5-2650 v4 @ 2.20GHz, 128Gb RAM while
> > CSV
Hi Anton,
Le 12/07/2019 à 23:21, Malakhov, Anton a écrit :
>
> The result is that all these execution nodes scale well enough and run under
> 100 milliseconds on my 2 x Xeon E5-2650 v4 @ 2.20GHz, 128Gb RAM while CSV
> reader takes several seconds to complete even reading from in-memory file
Hi, folks
We were discussing improvements for the threading engine back in May and agreed
to implement benchmarks (sorry, I've lost the original mail thread, here is the
link:
> From: Jed Brown [mailto:j...@jedbrown.org]
> Sent: Monday, May 6, 2019 16:35
> Nice paper, thanks! Did you investigate latency impact from the IPC counting
> semaphore? Is your test code available?
Not that deep. Basically I was looking only if its positive effect is enough to
overcome the
"Malakhov, Anton" writes:
> Jed,
>
>> From: Jed Brown [mailto:j...@jedbrown.org]
>> Sent: Friday, May 3, 2019 12:41
>
>> You linked to a NumPy discussion
>> (https://github.com/numpy/numpy/issues/11826) that is encountering the same
>> issues, but proposing solutions based on the global
Jed,
> From: Jed Brown [mailto:j...@jedbrown.org]
> Sent: Friday, May 3, 2019 12:41
> You linked to a NumPy discussion
> (https://github.com/numpy/numpy/issues/11826) that is encountering the same
> issues, but proposing solutions based on the global environment.
> That is perhaps acceptable for
> The question is whether you want to spend at least a month or more of
> intense development on something else (a basic query engine, as we've been
> discussing in [1]) before we are able to develop consensus about the
> approach to threading. Personally, I would not make this choice given that
>
Anton, per your comment:
> Sounds like a good way to go! We'll create a demo, as you suggested,
> implementing a parallel execution model for a simple analytics pipeline that
> reads and processes the files. My only concern is about adding more pipeline
> breaker nodes and compute intensive
"Malakhov, Anton" writes:
>> > the library creates threads internally. It's a disaster for managing
>> > oversubscription and affinity issues among groups of threads and/or
>> > multiple processes (e.g., MPI).
>
> This is exactly what I'm talking about referring as issues with threading
>
Le 03/05/2019 à 17:57, Jed Brown a écrit :
>
>>> The library is then free to use constructs like omp taskgroup/taskloop
>>> as granularity warrants; it will never utilize threads that the
>>> application didn't explicitly give it.
>>
>> I don't think we're planning to use OpenMP in Arrow,
Thanks for your answers,
> -Original Message-
> From: Antoine Pitrou [mailto:anto...@python.org]
> Sent: Friday, May 3, 2019 03:54
> Le 03/05/2019 à 05:47, Jed Brown a écrit :
> > I would caution to please not commit to the MKL/BLAS model in which
I'm actually talking about threading
Antoine Pitrou writes:
> Hi Jed,
>
> Le 03/05/2019 à 05:47, Jed Brown a écrit :
>> I would caution to please not commit to the MKL/BLAS model in which the
>> library creates threads internally. It's a disaster for managing
>> oversubscription and affinity issues among groups of threads and/or
Hi Jed,
Le 03/05/2019 à 05:47, Jed Brown a écrit :
> I would caution to please not commit to the MKL/BLAS model in which the
> library creates threads internally. It's a disaster for managing
> oversubscription and affinity issues among groups of threads and/or
> multiple processes (e.g.,
it is an acceptable way at all
> and what's the roadmap.
>
> Thanks,
> // Anton
>
>
> -Original Message-
> From: Wes McKinney [mailto:wesmck...@gmail.com]
> Sent: Thursday, May 2, 2019 13:52
> To: dev@arrow.apache.org
> Subject: Re: [DISCUSS][C++][Proposa
eve and to focus on this goal.
> Thus, I wanted to check with the community if it is an acceptable way at all
> and what's the roadmap.
>
> Thanks,
> // Anton
>
>
> -Original Message-
> From: Wes McKinney [mailto:wesmck...@gmail.com]
> Sent: Thursday, May 2, 2019 13:52
> To: de
and what's
the roadmap.
Thanks,
// Anton
-Original Message-
From: Wes McKinney [mailto:wesmck...@gmail.com]
Sent: Thursday, May 2, 2019 13:52
To: dev@arrow.apache.org
Subject: Re: [DISCUSS][C++][Proposal] Threading engine for Arrow
hi Anton,
Thank you for bringing your expertise
hi Anton,
Thank you for bringing your expertise to the project -- this is a very
useful discussion to have.
Partly why our threading capabilities in the project are not further
developed is that there is not much that needs to be parallelized. It
would be like designing a supercharger when you
Hi dear Arrow developers, Antoine,
I'd like to kick off the discussion of the threading engine that Arrow can use
underneath for implementing multicore parallelism for execution nodes, kernels,
and/or all the functions, which can be optimized this way.
I've documented some ideas on Arrow's
27 matches
Mail list logo