asynchronous and vectorized execution
Hi,
I realize that we haven't gotten 9.6beta1 out the door yet, but I
think we can't really wait much longer to start having at least some
discussion of 9.7 topics, so I'm going to go ahead and put this one
out there. I believe there are other people thinking about these
topics as well, including Andres Freund, Kyotaro Horiguchi, and
probably some folks at 2ndQuadrant (but I don't know exactly who). To
make a long story short, I think there are several different areas
where we should consider major upgrades to our executor. It's too
slow and it doesn't do everything we want it to do. The main things
on my mind are:
1. asynchronous execution, by which I mean the ability of a node to
somehow say that it will generate a tuple eventually, but is not yet
ready, so that the executor can go run some other part of the plan
tree while it waits. This case most obviously arises for foreign
tables, where it makes little sense to block on I/O if some other part
of the query tree could benefit from the CPU; consider SELECT * FROM
lt WHERE qual UNION SELECT * FROM ft WHERE qual. It is also a problem
for parallel query: in a parallel sequential scan, the next worker can
begin reading the next block even if the current block hasn't yet been
received from the OS. Whether or not this will be efficient is a
research question, but it can be done. However, imagine a parallel
scan of a btree index: we don't know what page to scan next until we
read the previous page and examine the next-pointer. In the meantime,
any worker that arrives at that scan node has no choice but to block.
It would be better if the scan node could instead say "hey, thanks for
coming but I'm really not ready to be on-CPU just at the moment" and
potentially allow the worker to go work in some other part of the
query tree. For that worker to actually find useful work to do
elsewhere, we'll probably need it to be the case either that the table
is partitioned or the original query will need to involve UNION ALL,
but those are not silly cases to worry about, particularly if we get
native partitioning in 9.7.
2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior, since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in
the first hash table, and then probe for all of those in the second
hash table, and so on. What we do instead is fetch one tuple and
probe for it in all 5 hash tables, and then repeat. If one of those
hash tables would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse. But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next
tuple from the probe table means calling a function which in turn
calls another function which probably calls another function which
probably calls another function and now about 4 layers down we
actually get the next tuple. If the scan returned a batch of tuples
to the hash join, fetching the next tuple from the batch would
probably be 0 or 1 function calls rather than ... more. Admittedly,
you've got to consider the cost of marshaling the batches but I'm
optimistic that there are cycles to be squeezed out here. We might
also want to consider storing batches of tuples in a column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.
Obviously, both of these are big projects that could touch a large
amount of executor code, and there may be other ideas, in addition to
these, which some of you may be thinking about that could also touch a
large amount of executor code. It would be nice to agree on a way
forward that minimizes code churn and maximizes everyone's attempt to
contribute without conflicting with each other. Also, it seems
desirable to enable, as far as possible, incremental development - in
particular, it seems to me that it would be good to pick a design that
doesn't require massive changes to every node all at once. A single
patch that adds some capability to every node in the executor in one
fell swoop is going to be too large to review effectively.
My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple. Similarly, for vectorized execution,
a node might return a bunch of tuples all at once. ExecProcNode will
extract the first one and return it to the caller, and subsequent
calls to ExecProcNode will iterate through the rest of the batch, only
calling the underlying node-specific function when the batch is
exhausted. In this way, code that doesn't know about the new stuff
can continue to work pretty much as it does today. Also, and I think
this is important, nodes don't need the permission of their parent
node to use these new capabilities. They can use them whenever they
wish, without worrying about whether the upper node is prepared to
deal with it. If not, ExecProcNode will paper over the problem. This
seems to me to be a good way to keep the code simple.
For asynchronous execution, I have gone so far as to mock up a bit of
what this might look like. This shouldn't be taken very seriously at
this point, but I'm attaching a few very-much-WIP patches to show the
direction of my line of thinking. Basically, I propose to have
ExecBlah (that is, ExecBitmapHeapScan, ExecAppend, etc.) return tuples
by putting them into a new PlanState member called "result", which is
just a Node * so that we can support multiple types of results,
instead of returning them. There is also a result_ready boolean, so
that a node can return without setting this Boolean to engage
asynchronous behavior. This triggers an "event loop", which
repeatedly waits for FDs chosen by waiting nodes to become readable
and/or writeable and then gives the node a chance to react.
Eventually, the waiting node will stop waiting and have a result
ready, at which point the event loop will give the parent of that node
a chance to run. If that node consequently becomes ready, then its
parent gets a chance to run. Eventually (we hope), the node for which
we're waiting becomes ready, and we can then read a result tuple.
With some more work, this seems like it can handle the FDW case, but I
haven't worked out how to make it handle the related parallel query
case. What we want there is to wait not for the readiness of an FD
but rather for some other process involved in the parallel query to
reach a point where it can welcome assistance executing that node. I
don't know exactly what the signaling for that should look like yet -
maybe setting the process latch or something.
By the way, one smaller executor project that I think we should also
look at has to do with this comment in nodeSeqScan.c:
static bool
SeqRecheck(SeqScanState *node, TupleTableSlot *slot)
{
/*
* Note that unlike IndexScan, SeqScan never use keys in heap_beginscan
* (and this is very bad) - so, here we do not check are keys ok or not.
*/
return true;
}
Some quick prototyping by my colleague Dilip Kumar suggests that, in
fact, there are cases where pushing down keys into heap_beginscan()
could be significantly faster. Some care is required here because any
functions we execute as scan keys are run with the buffer locked, so
we had better not run anything very complicated. But doing this for
simple things like integer equality operators seems like it could save
quite a few buffer lock/unlock cycles and some other executor overhead
as well.
Thoughts, ideas, suggestions, etc. very welcome.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
0001-Modify-PlanState-to-include-a-pointer-to-the-parent-.patchtext/x-diff; charset=US-ASCII; name=0001-Modify-PlanState-to-include-a-pointer-to-the-parent-.patchDownload+77-38
0002-Modify-PlanState-to-have-result-result_ready-fields.patchtext/x-diff; charset=US-ASCII; name=0002-Modify-PlanState-to-have-result-result_ready-fields.patchDownload+360-198
0003-Lightweight-framework-for-waiting-for-events.patchtext/x-diff; charset=US-ASCII; name=0003-Lightweight-framework-for-waiting-for-events.patchDownload+352-26
On 9 May 2016 at 19:33, Robert Haas <robertmhaas@gmail.com> wrote:
I believe there are other people thinking about these
topics as well, including Andres Freund, Kyotaro Horiguchi, and
probably some folks at 2ndQuadrant (but I don't know exactly who).
1. asynchronous execution
Not looking at that.
2. vectorized execution...
We might also want to consider storing batches of tuples in a
column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.
Team is about 2 years into research and coding prototype on those topics at
this point, with agreed time for further work over next 2 years.
I'll let my colleagues chime in with details since I'm not involved at that
level any more.
--
Simon Riggs http://www.2ndQuadrant.com/
<http://www.2ndquadrant.com/>
PostgreSQL Development, 24x7 Support, Remote DBA, Training & Services
On 10 May 2016 at 05:33, Robert Haas <robertmhaas@gmail.com> wrote:
2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior, since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in
the first hash table, and then probe for all of those in the second
hash table, and so on. What we do instead is fetch one tuple and
probe for it in all 5 hash tables, and then repeat. If one of those
hash tables would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse. But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next
tuple from the probe table means calling a function which in turn
calls another function which probably calls another function which
probably calls another function and now about 4 layers down we
actually get the next tuple. If the scan returned a batch of tuples
to the hash join, fetching the next tuple from the batch would
probably be 0 or 1 function calls rather than ... more. Admittedly,
you've got to consider the cost of marshaling the batches but I'm
optimistic that there are cycles to be squeezed out here. We might
also want to consider storing batches of tuples in a column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.
It's interesting that you mention this. We identified this as a pain
point during our work on column stores last year. Simply passing
single tuples around the executor is really unfriendly towards L1
instruction cache, plus also the points you mention about L3 cache and
hash tables and tuple stores. I really think that we're likely to see
significant gains by processing >1 tuple at a time, so this topic very
much interests me.
On researching this we've found that other peoples research does
indicate that there are gains to be had:
http://www.openlinksw.com/weblog/oerling/
In that blog there's a table that indicates that this row-store
database saw a 4.4x performance improvement from changing from a
tuple-at-a-time executor to a batch tuple executor.
Batch Size 1 tuple = 122 seconds
Batch Size 10k tuples = 27.7 seconds
When we start multiplying those increases with the increases with
something like parallel query then we're starting to see very nice
gains in performance.
Alvaro, Tomas and I had been discussing this and late last year I did
look into what would be required to allow this to happen in Postgres.
Basically there's 2 sub-projects, I'll describe what I've managed to
learn so far about each, and the rough plan that I have to implement
them:
1. Batch Execution:
a. Modify ScanAPI to allow batch tuple fetching in predefined batch sizes.
b. Modify TupleTableSlot to allow > 1 tuple to be stored. Add flag to
indicate if the struct contains a single or a multiple tuples.
Multiple tuples may need to be deformed in a non-lazy fashion in order
to prevent too many buffers from having to be pinned at once. Tuples
will be deformed into arrays of each column rather than arrays for
each tuple (this part is important to support the next sub-project)
c. Modify some nodes (perhaps start with nodeAgg.c) to allow them to
process a batch TupleTableSlot. This will require some tight loop to
aggregate the entire TupleTableSlot at once before returning.
d. Add function in execAmi.c which returns true or false depending on
if the node supports batch TupleTableSlots or not.
e. At executor startup determine if the entire plan tree supports
batch TupleTableSlots, if so enable batch scan mode.
That at least is my ideas for stage 1. There's still more to work out.
e.g should batch mode occur when the query has a LIMIT? we might not
want to waste time gather up extra tuples when we're just going to
stop after the first one. So perhaps 'e' above should be up to the
planner instead. Further development work here might add a node type
that de-batches a TupleTableSlot to allow nodes which don't support
batching to be in the plan, i.e "mixed execution mode". I'm less
excited about this as it may be difficult to cost that operation,
probably the time would be better spend just batch-enabling the other
node types, which *may* not be all that difficult. I'm also assuming
that batch mode (in all cases apart from queries with LIMIT or
cursors) will always be faster than tuple-at-a-time, so requires no
costings from the planner.
2. Vector processing
(I admit that I've given this part much less thought so far, but
here's what I have in mind)
This depends on batch execution, and is intended to allow the executor
to perform function calls to an entire batch at once, rather than
tuple-at-a-time. For example, let's take the following example;
SELECT a+b FROM t;
here (as of now) we'd scan "t" one row at a time and perform a+b after
having deformed enough of the tuple to do that. We'd then go and get
another Tuple from the scan node and repeat until the scan gave us no
more Tuples.
With batch execution we'd fetch multiple Tuples from the scan and we'd
then perform the call to say int4_pl() multiple times, which still
kinda sucks as it means calling int4_pl() possibly millions of times
(once per tuple). The vector mode here would require that we modify
pg_operator to add a vector function for each operator so that we can
call the function passing in an array of Datums and a length to have
SIMD operations perform the addition, so we'd call something like
int4_pl_vector() only once per batch of tuples allowing the CPU to
perform SIMD operations on those datum arrays. This could be done in
an incremental way as the code could just callback on the standard
function in cases where a vectorised version of it is not available.
Thought is needed here about when exactly this decision is made as the
user may not have permissions to execute the vector function, so it
can't simply be a run time check. These functions would simply return
another vector of the results. Aggregates could be given a vector
transition function, where something like COUNT(*)'s vector_transfn
would simply just current_count += vector_length;
This project does appear to require that we bloat the code with 100's
of vector versions of each function. I'm not quite sure if there's a
better way to handle this. The problem is that the fmgr is pretty much
a barrier to SIMD operations, and this was the only idea that I've had
so far about breaking through that barrier. So further ideas here are
very welcome.
The idea here is that these 2 projects help pave the way to bring
columnar storage into PostgreSQL. Without these we're unlikely to get
much benefit of columnar storage as we'd be stuck processing rows one
at a time still. Adding columnar storage on the top of the above
should further increase performance as we can skip the tuple-deform
step and pull columnar array/vectors directly into a TupleTableSlot,
although some trickery would be involved here when the scan has keys.
I just want to add that both of the above do require more thought. We
realised that this was required quite late in our column store work
(which we've all now taken a break from to work on other things), so
we've had little time to look much further into it. Although I should
be starting work again on this in the next few months in the hopes to
have something, even the most simple version of it in 9.7.
Comments are welcome
--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
-----Original Message-----
From: pgsql-hackers-owner@postgresql.org
[mailto:pgsql-hackers-owner@postgresql.org] On Behalf Of Robert Haas
Sent: Tuesday, May 10, 2016 2:34 AM
To: pgsql-hackers@postgresql.org
Subject: [HACKERS] asynchronous and vectorized executionHi,
I realize that we haven't gotten 9.6beta1 out the door yet, but I
think we can't really wait much longer to start having at least some
discussion of 9.7 topics, so I'm going to go ahead and put this one
out there. I believe there are other people thinking about these
topics as well, including Andres Freund, Kyotaro Horiguchi, and
probably some folks at 2ndQuadrant (but I don't know exactly who). To
make a long story short, I think there are several different areas
where we should consider major upgrades to our executor. It's too
slow and it doesn't do everything we want it to do. The main things
on my mind are:1. asynchronous execution, by which I mean the ability of a node to
somehow say that it will generate a tuple eventually, but is not yet
ready, so that the executor can go run some other part of the plan
tree while it waits. This case most obviously arises for foreign
tables, where it makes little sense to block on I/O if some other part
of the query tree could benefit from the CPU; consider SELECT * FROM
lt WHERE qual UNION SELECT * FROM ft WHERE qual. It is also a problem
for parallel query: in a parallel sequential scan, the next worker can
begin reading the next block even if the current block hasn't yet been
received from the OS. Whether or not this will be efficient is a
research question, but it can be done. However, imagine a parallel
scan of a btree index: we don't know what page to scan next until we
read the previous page and examine the next-pointer. In the meantime,
any worker that arrives at that scan node has no choice but to block.
It would be better if the scan node could instead say "hey, thanks for
coming but I'm really not ready to be on-CPU just at the moment" and
potentially allow the worker to go work in some other part of the
query tree. For that worker to actually find useful work to do
elsewhere, we'll probably need it to be the case either that the table
is partitioned or the original query will need to involve UNION ALL,
but those are not silly cases to worry about, particularly if we get
native partitioning in 9.7.
Is the parallel aware Append node sufficient to run multiple nodes
asynchronously? (Sorry, I couldn't have enough time to code the feature
even though we had discussion before.)
If a part of child-nodes are blocked by I/O or other heavy stuff, it
cannot enqueue the results into shm_mq, thus, Gather node naturally
skip nodes that are not ready.
In the above example, scan on foreign-table takes longer lead time than
local scan. If Append can map every child nodes on individual workers,
local scan worker begins to return tuples at first, then, mixed tuples
shall be returned eventually.
However, the process internal asynchronous execution may be also beneficial
in case when cost of shm_mq is not ignorable (e.g, no scan qualifiers
are given to worker process). I think it allows to implement pre-fetching
very naturally.
2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior,
My concern about ExecProcNode is, it is constructed with a large switch
... case statement. It involves tons of comparison operation at run-time.
If we replace this switch ... case by function pointer, probably, it make
performance improvement. Especially, OLAP workloads that process large
amount of rows.
since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in
the first hash table, and then probe for all of those in the second
hash table, and so on. What we do instead is fetch one tuple and
probe for it in all 5 hash tables, and then repeat. If one of those
hash tables would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse.
I can agree with the above concern from my experience. Each HashJoin
step needs to fill up TupleTableSlot for each depth. Mostly, it is
just relocation of the attributes in case of multi-tables joins.
If HashJoin could gather five underlying hash-tables at once, it can
reduce unnecessary setup of intermediation tuples.
A position example is GpuHashJoin in PG-Strom. It constructs multi-
relations hash table, then, produce joined tuples at once.
Its performance is generally good.
But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next
tuple from the probe table means calling a function which in turn
calls another function which probably calls another function which
probably calls another function and now about 4 layers down we
actually get the next tuple. If the scan returned a batch of tuples
to the hash join, fetching the next tuple from the batch would
probably be 0 or 1 function calls rather than ... more. Admittedly,
you've got to consider the cost of marshaling the batches but I'm
optimistic that there are cycles to be squeezed out here. We might
also want to consider storing batches of tuples in a column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.Obviously, both of these are big projects that could touch a large
amount of executor code, and there may be other ideas, in addition to
these, which some of you may be thinking about that could also touch a
large amount of executor code. It would be nice to agree on a way
forward that minimizes code churn and maximizes everyone's attempt to
contribute without conflicting with each other. Also, it seems
desirable to enable, as far as possible, incremental development - in
particular, it seems to me that it would be good to pick a design that
doesn't require massive changes to every node all at once. A single
patch that adds some capability to every node in the executor in one
fell swoop is going to be too large to review effectively.My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple.
Backward compatibility is good. In addition, child node may want to
know the context when it is called. It may want to switch the behavior
according to the caller's expectation. For example, it may be beneficial
if SeqScan makes more aggressive prefetching on asynchronous execution.
Also, can we consider which data format will be returned from the child
node during the planning stage? It affects to the cost of inter-node
data exchange. If a pair of parent-node and child-node supports its
special data format (like as existing HashJoin and Hash doing), it shall
be a discount factor of cost estimation.
Similarly, for vectorized execution,
a node might return a bunch of tuples all at once. ExecProcNode will
extract the first one and return it to the caller, and subsequent
calls to ExecProcNode will iterate through the rest of the batch, only
calling the underlying node-specific function when the batch is
exhausted. In this way, code that doesn't know about the new stuff
can continue to work pretty much as it does today. Also, and I think
this is important, nodes don't need the permission of their parent
node to use these new capabilities. They can use them whenever they
wish, without worrying about whether the upper node is prepared to
deal with it. If not, ExecProcNode will paper over the problem. This
seems to me to be a good way to keep the code simple.For asynchronous execution, I have gone so far as to mock up a bit of
what this might look like. This shouldn't be taken very seriously at
this point, but I'm attaching a few very-much-WIP patches to show the
direction of my line of thinking. Basically, I propose to have
ExecBlah (that is, ExecBitmapHeapScan, ExecAppend, etc.) return tuples
by putting them into a new PlanState member called "result", which is
just a Node * so that we can support multiple types of results,
instead of returning them. There is also a result_ready boolean, so
that a node can return without setting this Boolean to engage
asynchronous behavior. This triggers an "event loop", which
repeatedly waits for FDs chosen by waiting nodes to become readable
and/or writeable and then gives the node a chance to react.
Eventually, the waiting node will stop waiting and have a result
ready, at which point the event loop will give the parent of that node
a chance to run. If that node consequently becomes ready, then its
parent gets a chance to run. Eventually (we hope), the node for which
we're waiting becomes ready, and we can then read a result tuple.
With some more work, this seems like it can handle the FDW case, but I
haven't worked out how to make it handle the related parallel query
case. What we want there is to wait not for the readiness of an FD
but rather for some other process involved in the parallel query to
reach a point where it can welcome assistance executing that node. I
don't know exactly what the signaling for that should look like yet -
maybe setting the process latch or something.By the way, one smaller executor project that I think we should also
look at has to do with this comment in nodeSeqScan.c:static bool
SeqRecheck(SeqScanState *node, TupleTableSlot *slot)
{
/*
* Note that unlike IndexScan, SeqScan never use keys in heap_beginscan
* (and this is very bad) - so, here we do not check are keys ok or not.
*/
return true;
}Some quick prototyping by my colleague Dilip Kumar suggests that, in
fact, there are cases where pushing down keys into heap_beginscan()
could be significantly faster. Some care is required here because any
functions we execute as scan keys are run with the buffer locked, so
we had better not run anything very complicated. But doing this for
simple things like integer equality operators seems like it could save
quite a few buffer lock/unlock cycles and some other executor overhead
as well.Thoughts, ideas, suggestions, etc. very welcome.
Thanks,
--
NEC Business Creation Division / PG-Strom Project
KaiGai Kohei <kaigai@ak.jp.nec.com>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 9 May 2016 8:34 pm, "David Rowley" <david.rowley@2ndquadrant.com> wrote:
This project does appear to require that we bloat the code with 100's
of vector versions of each function. I'm not quite sure if there's a
better way to handle this. The problem is that the fmgr is pretty much
a barrier to SIMD operations, and this was the only idea that I've had
so far about breaking through that barrier. So further ideas here are
very welcome.
Well yes and no. In practice I think you only need to worry about
vectorised versions of integer and possibly float. For other data types
there either aren't vectorised operators or there's little using them.
And I'll make a bold claim here that the only operators I think really
matter are =
The rain is because using SIMD instructions is a minor win if you have any
further work to do per tuple. The only time it's a big win is if you're
eliminating entire tuples from consideration efficiently. = is going to do
that often, other btree operator classes might be somewhat useful, but
things like + really only would come up in odd examples.
But even that understates things. If you have column oriented storage then
= becomes even more important since every scan has a series of implied
equijoins to reconstruct the tuple. And the coup de grace is that in a
column oriented storage you try to store variable length data as integer
indexes into a dictionary of common values so *everything* is an integer =
operation.
How to do this without punching right through the executor as an
abstraction and still supporting extensible data types and operators was
puzzling me already. I do think it involves having these vector operators
in the catalogue and also some kind of compression mapping to integer
indexes. But I'm not sure that's all that would be needed.
On 10 May 2016 at 13:38, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
My concern about ExecProcNode is, it is constructed with a large switch
... case statement. It involves tons of comparison operation at run-time.
If we replace this switch ... case by function pointer, probably, it make
performance improvement. Especially, OLAP workloads that process large
amount of rows.
I imagined that any decent compiler would have built the code to use
jump tables for this. I have to say that I've never checked to make
sure though.
--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
-----Original Message-----
From: pgsql-hackers-owner@postgresql.org
[mailto:pgsql-hackers-owner@postgresql.org] On Behalf Of David Rowley
Sent: Tuesday, May 10, 2016 2:01 PM
To: Kaigai Kouhei(海外 浩平)
Cc: Robert Haas; pgsql-hackers@postgresql.org
Subject: Re: [HACKERS] asynchronous and vectorized executionOn 10 May 2016 at 13:38, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
My concern about ExecProcNode is, it is constructed with a large switch
... case statement. It involves tons of comparison operation at run-time.
If we replace this switch ... case by function pointer, probably, it make
performance improvement. Especially, OLAP workloads that process large
amount of rows.I imagined that any decent compiler would have built the code to use
jump tables for this. I have to say that I've never checked to make
sure though.
Ah, indeed, you are right. Please forget above part.
In GCC 4.8.5, the case label between T_ResultState and T_LimitState were
handled using jump table.
TupleTableSlot *
ExecProcNode(PlanState *node)
{
:
<snip>
:
switch (nodeTag(node))
5ad361: 8b 03 mov (%rbx),%eax
5ad363: 2d c9 00 00 00 sub $0xc9,%eax
5ad368: 83 f8 24 cmp $0x24,%eax
5ad36b: 0f 87 4f 02 00 00 ja 5ad5c0 <ExecProcNode+0x290>
5ad371: ff 24 c5 68 48 8b 00 jmpq *0x8b4868(,%rax,8)
5ad378: 0f 1f 84 00 00 00 00 nopl 0x0(%rax,%rax,1)
5ad37f: 00
Thanks,
--
NEC Business Creation Division / PG-Strom Project
KaiGai Kohei <kaigai@ak.jp.nec.com>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10 May 2016 at 16:34, Greg Stark <stark@mit.edu> wrote:
On 9 May 2016 8:34 pm, "David Rowley" <david.rowley@2ndquadrant.com> wrote:
This project does appear to require that we bloat the code with 100's
of vector versions of each function. I'm not quite sure if there's a
better way to handle this. The problem is that the fmgr is pretty much
a barrier to SIMD operations, and this was the only idea that I've had
so far about breaking through that barrier. So further ideas here are
very welcome.Well yes and no. In practice I think you only need to worry about vectorised
versions of integer and possibly float. For other data types there either
aren't vectorised operators or there's little using them.And I'll make a bold claim here that the only operators I think really
matter are =The rain is because using SIMD instructions is a minor win if you have any
further work to do per tuple. The only time it's a big win is if you're
eliminating entire tuples from consideration efficiently. = is going to do
that often, other btree operator classes might be somewhat useful, but
things like + really only would come up in odd examples.But even that understates things. If you have column oriented storage then =
becomes even more important since every scan has a series of implied
equijoins to reconstruct the tuple. And the coup de grace is that in a
column oriented storage you try to store variable length data as integer
indexes into a dictionary of common values so *everything* is an integer =
operation.How to do this without punching right through the executor as an abstraction
and still supporting extensible data types and operators was puzzling me
already. I do think it involves having these vector operators in the
catalogue and also some kind of compression mapping to integer indexes. But
I'm not sure that's all that would be needed.
Perhaps the first move to make on this front will be for aggregate
functions. Experimentation might be quite simple to realise which
functions will bring enough benefit. I imagined that even Datums where
the type is not processor native might yield a small speedup, not from
SIMD, but just from less calls through fmgr. Perhaps we'll realise
that those are not worth the trouble, I've no idea at this stage.
--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
2016-05-10 8:05 GMT+02:00 David Rowley <david.rowley@2ndquadrant.com>:
On 10 May 2016 at 16:34, Greg Stark <stark@mit.edu> wrote:
On 9 May 2016 8:34 pm, "David Rowley" <david.rowley@2ndquadrant.com>
wrote:
This project does appear to require that we bloat the code with 100's
of vector versions of each function. I'm not quite sure if there's a
better way to handle this. The problem is that the fmgr is pretty much
a barrier to SIMD operations, and this was the only idea that I've had
so far about breaking through that barrier. So further ideas here are
very welcome.Well yes and no. In practice I think you only need to worry about
vectorised
versions of integer and possibly float. For other data types there either
aren't vectorised operators or there's little using them.And I'll make a bold claim here that the only operators I think really
matter are =The rain is because using SIMD instructions is a minor win if you have
any
further work to do per tuple. The only time it's a big win is if you're
eliminating entire tuples from consideration efficiently. = is going todo
that often, other btree operator classes might be somewhat useful, but
things like + really only would come up in odd examples.But even that understates things. If you have column oriented storage
then =
becomes even more important since every scan has a series of implied
equijoins to reconstruct the tuple. And the coup de grace is that in a
column oriented storage you try to store variable length data as integer
indexes into a dictionary of common values so *everything* is an integer=
operation.
How to do this without punching right through the executor as an
abstraction
and still supporting extensible data types and operators was puzzling me
already. I do think it involves having these vector operators in the
catalogue and also some kind of compression mapping to integer indexes.But
I'm not sure that's all that would be needed.
Perhaps the first move to make on this front will be for aggregate
functions. Experimentation might be quite simple to realise which
functions will bring enough benefit. I imagined that even Datums where
the type is not processor native might yield a small speedup, not from
SIMD, but just from less calls through fmgr. Perhaps we'll realise
that those are not worth the trouble, I've no idea at this stage.
It can be reduced to sum and count in first iteration. On other hand lot of
OLAP reports is based on pretty complex expressions - and there probably
the compilation is better way.
Regards
Pavel
Show quoted text
--
David Rowley http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Training & Services--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hi,
1. asynchronous execution,
It seems to me that asynchronous execution can be considered as alternative to multithreading model (in case of PostgreSQL the roles of threads are played by workers).
Async. operations are used to have smaller overhead but have scalability problems (because in most implementation of cooperative multitasking there is just one processing thread and so it can not consume more than one core).
So I wonder whether asynchronous execution is trying to achieve that same goal as parallel query execution but using slightly different mechanism.
You wrote:
in the meantime, any worker that arrives at that scan node has no choice but to block.
What's wrong with it that worker is blocked? You can just have more workers (more than CPU cores) to let other of them continue to do useful work.
But I agree that
Whether or not this will be efficient is a research question
2. vectorized execution
Vector IO is very important for columnar store. In IMCS extension (in-memory columnar store) using vector operations allows to increase speed 10-100 times depending on size of data set and query. Obviously the best results are for grand aggregates.
But there are some researches, for example:
http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
showing that the same or even better effect can be achieved by generation native code for query execution plan (which is not so difficult now, thanks to LLVM).
It eliminates interpretation overhead and increase cache locality.
I get similar results with my own experiments of accelerating SparkSQL. Instead of native code generation I used conversion of query plans to C code and experiment with different data representation. "Horisontal model" with loading columns on demands shows better performance than columnar store.
As far as I know native code generator is currently developed for PostgreSQL by ISP RAN
Sorry, slides in Russian:
https://pgconf.ru/media/2016/02/19/6%20Мельник%20Дмитрий%20Михайлович,%2005-02-2016.pdf
At this moment (February) them have implemented translation of only few PostgreSQL operators used by ExecQuals and do not support aggregates.
Them get about 2 times increase of speed at synthetic queries and 25% increase at TPC-H Q1 (for Q1 most critical is generation of native code for aggregates, because ExecQual itself takes only 6% of time for this query).
Actually these 25% for Q1 were achieved not by using dynamic code generation, but switching from PULL to PUSH model in executor.
It seems to be yet another interesting PostgreSQL executor transformation.
As far as I know, them are going to publish result of their work to open source...
On May 9, 2016, at 8:33 PM, Robert Haas wrote:
Show quoted text
Hi,
I realize that we haven't gotten 9.6beta1 out the door yet, but I
think we can't really wait much longer to start having at least some
discussion of 9.7 topics, so I'm going to go ahead and put this one
out there. I believe there are other people thinking about these
topics as well, including Andres Freund, Kyotaro Horiguchi, and
probably some folks at 2ndQuadrant (but I don't know exactly who). To
make a long story short, I think there are several different areas
where we should consider major upgrades to our executor. It's too
slow and it doesn't do everything we want it to do. The main things
on my mind are:1. asynchronous execution, by which I mean the ability of a node to
somehow say that it will generate a tuple eventually, but is not yet
ready, so that the executor can go run some other part of the plan
tree while it waits. This case most obviously arises for foreign
tables, where it makes little sense to block on I/O if some other part
of the query tree could benefit from the CPU; consider SELECT * FROM
lt WHERE qual UNION SELECT * FROM ft WHERE qual. It is also a problem
for parallel query: in a parallel sequential scan, the next worker can
begin reading the next block even if the current block hasn't yet been
received from the OS. Whether or not this will be efficient is a
research question, but it can be done. However, imagine a parallel
scan of a btree index: we don't know what page to scan next until we
read the previous page and examine the next-pointer. In the meantime,
any worker that arrives at that scan node has no choice but to block.
It would be better if the scan node could instead say "hey, thanks for
coming but I'm really not ready to be on-CPU just at the moment" and
potentially allow the worker to go work in some other part of the
query tree. For that worker to actually find useful work to do
elsewhere, we'll probably need it to be the case either that the table
is partitioned or the original query will need to involve UNION ALL,
but those are not silly cases to worry about, particularly if we get
native partitioning in 9.7.2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior, since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in
the first hash table, and then probe for all of those in the second
hash table, and so on. What we do instead is fetch one tuple and
probe for it in all 5 hash tables, and then repeat. If one of those
hash tables would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse. But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next
tuple from the probe table means calling a function which in turn
calls another function which probably calls another function which
probably calls another function and now about 4 layers down we
actually get the next tuple. If the scan returned a batch of tuples
to the hash join, fetching the next tuple from the batch would
probably be 0 or 1 function calls rather than ... more. Admittedly,
you've got to consider the cost of marshaling the batches but I'm
optimistic that there are cycles to be squeezed out here. We might
also want to consider storing batches of tuples in a column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.Obviously, both of these are big projects that could touch a large
amount of executor code, and there may be other ideas, in addition to
these, which some of you may be thinking about that could also touch a
large amount of executor code. It would be nice to agree on a way
forward that minimizes code churn and maximizes everyone's attempt to
contribute without conflicting with each other. Also, it seems
desirable to enable, as far as possible, incremental development - in
particular, it seems to me that it would be good to pick a design that
doesn't require massive changes to every node all at once. A single
patch that adds some capability to every node in the executor in one
fell swoop is going to be too large to review effectively.My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple. Similarly, for vectorized execution,
a node might return a bunch of tuples all at once. ExecProcNode will
extract the first one and return it to the caller, and subsequent
calls to ExecProcNode will iterate through the rest of the batch, only
calling the underlying node-specific function when the batch is
exhausted. In this way, code that doesn't know about the new stuff
can continue to work pretty much as it does today. Also, and I think
this is important, nodes don't need the permission of their parent
node to use these new capabilities. They can use them whenever they
wish, without worrying about whether the upper node is prepared to
deal with it. If not, ExecProcNode will paper over the problem. This
seems to me to be a good way to keep the code simple.For asynchronous execution, I have gone so far as to mock up a bit of
what this might look like. This shouldn't be taken very seriously at
this point, but I'm attaching a few very-much-WIP patches to show the
direction of my line of thinking. Basically, I propose to have
ExecBlah (that is, ExecBitmapHeapScan, ExecAppend, etc.) return tuples
by putting them into a new PlanState member called "result", which is
just a Node * so that we can support multiple types of results,
instead of returning them. There is also a result_ready boolean, so
that a node can return without setting this Boolean to engage
asynchronous behavior. This triggers an "event loop", which
repeatedly waits for FDs chosen by waiting nodes to become readable
and/or writeable and then gives the node a chance to react.
Eventually, the waiting node will stop waiting and have a result
ready, at which point the event loop will give the parent of that node
a chance to run. If that node consequently becomes ready, then its
parent gets a chance to run. Eventually (we hope), the node for which
we're waiting becomes ready, and we can then read a result tuple.
With some more work, this seems like it can handle the FDW case, but I
haven't worked out how to make it handle the related parallel query
case. What we want there is to wait not for the readiness of an FD
but rather for some other process involved in the parallel query to
reach a point where it can welcome assistance executing that node. I
don't know exactly what the signaling for that should look like yet -
maybe setting the process latch or something.By the way, one smaller executor project that I think we should also
look at has to do with this comment in nodeSeqScan.c:static bool
SeqRecheck(SeqScanState *node, TupleTableSlot *slot)
{
/*
* Note that unlike IndexScan, SeqScan never use keys in heap_beginscan
* (and this is very bad) - so, here we do not check are keys ok or not.
*/
return true;
}Some quick prototyping by my colleague Dilip Kumar suggests that, in
fact, there are cases where pushing down keys into heap_beginscan()
could be significantly faster. Some care is required here because any
functions we execute as scan keys are run with the buffer locked, so
we had better not run anything very complicated. But doing this for
simple things like integer equality operators seems like it could save
quite a few buffer lock/unlock cycles and some other executor overhead
as well.Thoughts, ideas, suggestions, etc. very welcome.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
<0001-Modify-PlanState-to-include-a-pointer-to-the-parent-.patch><0002-Modify-PlanState-to-have-result-result_ready-fields.patch><0003-Lightweight-framework-for-waiting-for-events.patch>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Hello.
At Mon, 9 May 2016 13:33:55 -0400, Robert Haas <robertmhaas@gmail.com> wrote in <CA+Tgmobx8su_bYtAa3DgrqB+R7xZG6kHRj0ccMUUshKAQVftww@mail.gmail.com>
Hi,
I realize that we haven't gotten 9.6beta1 out the door yet, but I
think we can't really wait much longer to start having at least some
discussion of 9.7 topics, so I'm going to go ahead and put this one
out there. I believe there are other people thinking about these
topics as well, including Andres Freund, Kyotaro Horiguchi, and
probably some folks at 2ndQuadrant (but I don't know exactly who). To
make a long story short, I think there are several different areas
where we should consider major upgrades to our executor. It's too
slow and it doesn't do everything we want it to do. The main things
on my mind are:
1. asynchronous execution, by which I mean the ability of a node to
somehow say that it will generate a tuple eventually, but is not yet
ready, so that the executor can go run some other part of the plan
tree while it waits. This case most obviously arises for foreign
tables, where it makes little sense to block on I/O if some other part
of the query tree could benefit from the CPU; consider SELECT * FROM
lt WHERE qual UNION SELECT * FROM ft WHERE qual.
This is my main concern and what I wanted to solve.
It is also a problem
for parallel query: in a parallel sequential scan, the next worker can
begin reading the next block even if the current block hasn't yet been
received from the OS. Whether or not this will be efficient is a
research question, but it can be done. However, imagine a parallel
scan of a btree index: we don't know what page to scan next until we
read the previous page and examine the next-pointer. In the meantime,
any worker that arrives at that scan node has no choice but to block.
It would be better if the scan node could instead say "hey, thanks for
coming but I'm really not ready to be on-CPU just at the moment" and
potentially allow the worker to go work in some other part of the
query tree.
Especially for foreign tables, there must be gaps between sending
FETCH and getting the result. Visiting other tables is very
effective to fill the gaps. Using file descriptors is greatly
helps this in effective way, thanks to the new API
WaitEventSet. The attached is a WiP of PoC (sorry for including
some debug code and irrelevant code) of that. It is a bit
different in Exec* APIs from the 0002 patch but works even only
for postgres-fdw and append. It embeds waiting code into
ExecAppend but easily replaceable with the framework in the
Robert's 0003 patch.
Apart from the core part, for postgres-fdw, some scans resides
together on one connection. These scans share the same FD but
there's no means to identify for which scan-node the fd is
signalled. To handle the situation, we might need 'seemed to be
ready but really not' route.
For that worker to actually find useful work to do
elsewhere, we'll probably need it to be the case either that the table
is partitioned or the original query will need to involve UNION ALL,
but those are not silly cases to worry about, particularly if we get
native partitioning in 9.7.
One annoyance of this method is one FD with latch-like data
drain. Since we should provide FDs for such nodes, gather would
may have another data-passing channel on the FDs.
And I want to realize early-execution of async nodes. This might
need that all types of node return 'not-ready' for the first call
even if it is async-capable.
2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior, since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in
the first hash table, and then probe for all of those in the second
hash table, and so on. What we do instead is fetch one tuple and
probe for it in all 5 hash tables, and then repeat. If one of those
hash tables would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse. But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next
tuple from the probe table means calling a function which in turn
calls another function which probably calls another function which
probably calls another function and now about 4 layers down we
actually get the next tuple. If the scan returned a batch of tuples
to the hash join, fetching the next tuple from the batch would
probably be 0 or 1 function calls rather than ... more. Admittedly,
you've got to consider the cost of marshaling the batches but I'm
optimistic that there are cycles to be squeezed out here. We might
also want to consider storing batches of tuples in a column-optimized
rather than row-optimized format so that iterating through one or two
attributes across every tuple in the batch touches the minimal number
of cache lines.Obviously, both of these are big projects that could touch a large
amount of executor code, and there may be other ideas, in addition to
these, which some of you may be thinking about that could also touch a
large amount of executor code. It would be nice to agree on a way
forward that minimizes code churn and maximizes everyone's attempt to
contribute without conflicting with each other. Also, it seems
desirable to enable, as far as possible, incremental development - in
particular, it seems to me that it would be good to pick a design that
doesn't require massive changes to every node all at once. A single
patch that adds some capability to every node in the executor in one
fell swoop is going to be too large to review effectively.My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple. Similarly, for vectorized execution,
a node might return a bunch of tuples all at once. ExecProcNode will
extract the first one and return it to the caller, and subsequent
calls to ExecProcNode will iterate through the rest of the batch, only
calling the underlying node-specific function when the batch is
exhausted. In this way, code that doesn't know about the new stuff
can continue to work pretty much as it does today. Also, and I think
this is important, nodes don't need the permission of their parent
node to use these new capabilities. They can use them whenever they
wish, without worrying about whether the upper node is prepared to
deal with it. If not, ExecProcNode will paper over the problem. This
seems to me to be a good way to keep the code simple.
Agreed to returning not-ready state and wrapping nodes to
disguise old-style API, but I suppose Exec* may return a tuple as
it does corrently.
For asynchronous execution, I have gone so far as to mock up a bit of
what this might look like. This shouldn't be taken very seriously at
this point, but I'm attaching a few very-much-WIP patches to show the
direction of my line of thinking. Basically, I propose to have
ExecBlah (that is, ExecBitmapHeapScan, ExecAppend, etc.) return tuples
by putting them into a new PlanState member called "result", which is
just a Node * so that we can support multiple types of results,
instead of returning them. There is also a result_ready boolean, so
that a node can return without setting this Boolean to engage
asynchronous behavior. This triggers an "event loop", which
repeatedly waits for FDs chosen by waiting nodes to become readable
and/or writeable and then gives the node a chance to react.
Eventually, the waiting node will stop waiting and have a result
ready, at which point the event loop will give the parent of that node
a chance to run. If that node consequently becomes ready, then its
parent gets a chance to run. Eventually (we hope), the node for which
we're waiting becomes ready, and we can then read a result tuple.
I thought almost the same, even only for AppendNode..
With some more work, this seems like it can handle the FDW case, but I
haven't worked out how to make it handle the related parallel query
case. What we want there is to wait not for the readiness of an FD
but rather for some other process involved in the parallel query to
reach a point where it can welcome assistance executing that node. I
don't know exactly what the signaling for that should look like yet -
maybe setting the process latch or something.
Agreed as described above.
By the way, one smaller executor project that I think we should also
look at has to do with this comment in nodeSeqScan.c:static bool
SeqRecheck(SeqScanState *node, TupleTableSlot *slot)
{
/*
* Note that unlike IndexScan, SeqScan never use keys in heap_beginscan
* (and this is very bad) - so, here we do not check are keys ok or not.
*/
return true;
}Some quick prototyping by my colleague Dilip Kumar suggests that, in
fact, there are cases where pushing down keys into heap_beginscan()
could be significantly faster. Some care is required here because any
functions we execute as scan keys are run with the buffer locked, so
we had better not run anything very complicated. But doing this for
simple things like integer equality operators seems like it could save
quite a few buffer lock/unlock cycles and some other executor overhead
as well.
The cost of pushing-down keys on seqscans seems calucalatable
with a maybe-small amount of computation. So I suppose it is
promising.
Thoughts, ideas, suggestions, etc. very welcome.
regards,
--
Kyotaro Horiguchi
NTT Open Source Software Center
Attachments:
PoC-async-exec-horiguchi-20160510.difftext/x-patch; charset=us-asciiDownload+399-100
On 09 May 2016 23:04, Robert Haas Wrote:
2. vectorized execution, by which I mean the ability of a node to return
tuples in batches rather than one by one. Andres has opined more than
once that repeated trips through ExecProcNode defeat the ability of the
CPU to do branch prediction correctly, slowing the whole system down,
and that they also result in poor CPU cache behavior, since we jump all
over the place executing a little bit of code from each node before
moving onto the next rather than running one bit of code first, and then
another later. I think that's
probably right. For example, consider a 5-table join where all of
the joins are implemented as hash tables. If this query plan is going
to be run to completion, it would make much more sense to fetch, say,
100 tuples from the driving scan and then probe for all of those in the
first hash table, and then probe for all of those in the second hash
table, and so on. What we do instead is fetch one tuple and probe for
it in all 5 hash tables, and then repeat. If one of those hash tables
would fit in the CPU cache but all five together will not,
that seems likely to be a lot worse. But even just ignoring the CPU
cache aspect of it for a minute, suppose you want to write a loop to
perform a hash join. The inner loop fetches the next tuple from the
probe table and does a hash lookup. Right now, fetching the next tuple
from the probe table means calling a function which in turn calls
another function which probably calls another function which probably
calls another function and now about 4 layers down we actually get the
next tuple. If the scan returned a batch of tuples to the hash join,
fetching the next tuple from the batch would probably be 0 or 1 function
calls rather than ... more. Admittedly, you've got to consider the cost
of marshaling the batches but I'm optimistic that there are cycles to be
squeezed out here. We might also want to consider storing batches of
tuples in a column-optimized rather than row-optimized format so that
iterating through one or two attributes across every tuple in the batch
touches the minimal number of cache lines.
This sounds to be really great idea in the direction of performance improvement.
I would like to share my thought as per our research work in the similar area (Mostly it may be as you have mentioned).
Our goal with this work was to:
1. Makes the processing data-centric instead of operator centric.
2. Instead of pulling each tuple from immediate operator, operator can push the tuple to its parent. It can be allowed to push until it sees any operator, which cannot be processed without result from other operator.
3. Above two points to make better data-locality.
e.g. we had done some quick prototyping (take it just as thought provoker) as mentioned below:
Query: select * from tbl1, tbl2, tbl3 where tbl1.a=tbl2.b and tbl2.b=tbl3.c;
For hash join:
For each tuple t2 of tbl2
Materialize a hash tbl1.a = tbl2.b
For each tuple t3 of tbl3
Materialize a hash tbl2.b = tbl3.c
for each tuple t1 of tbl1
Search in hash tbl1.a = tbl2.b
Search in hash tbl2.b = tbl3.c
Output t1*t2*t3
Off course at each level if there is any additional Qual for the table, same can be applied.
Similarly for Nested Loop Join, plan tree can be processed in the form of post-order traversal of tree.
Scan first relation (leftmost relation), store all tuple --> Outer
Loop through all scan (Or some part of total tuples)node relation starting from second one
Scan the current relation
For each tuple, match with all tuples of outer result, build the combined tuple.
Save all combined satisfied tuple --> Outer
The result we got was really impressive.
There is a paper by Thomas Neumann on this idea: http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
Note: VitesseDB has also implemented this approach for Hash Join along with compilation and their result is really impressive.
Thanks and Regards,
Kumar Rajeev Rastogi.
http://rajeevrastogi.blogspot.in/
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, May 9, 2016 at 8:34 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:
It's interesting that you mention this. We identified this as a pain
point during our work on column stores last year. Simply passing
single tuples around the executor is really unfriendly towards L1
instruction cache, plus also the points you mention about L3 cache and
hash tables and tuple stores. I really think that we're likely to see
significant gains by processing >1 tuple at a time, so this topic very
much interests me.
Cool. I hope we can work together on it, and with anyone else who is
interested.
When we start multiplying those increases with the increases with
something like parallel query then we're starting to see very nice
gains in performance.
Check.
Alvaro, Tomas and I had been discussing this and late last year I did
look into what would be required to allow this to happen in Postgres.
Basically there's 2 sub-projects, I'll describe what I've managed to
learn so far about each, and the rough plan that I have to implement
them:1. Batch Execution:
a. Modify ScanAPI to allow batch tuple fetching in predefined batch sizes.
b. Modify TupleTableSlot to allow > 1 tuple to be stored. Add flag to
indicate if the struct contains a single or a multiple tuples.
Multiple tuples may need to be deformed in a non-lazy fashion in order
to prevent too many buffers from having to be pinned at once. Tuples
will be deformed into arrays of each column rather than arrays for
each tuple (this part is important to support the next sub-project)
c. Modify some nodes (perhaps start with nodeAgg.c) to allow them to
process a batch TupleTableSlot. This will require some tight loop to
aggregate the entire TupleTableSlot at once before returning.
d. Add function in execAmi.c which returns true or false depending on
if the node supports batch TupleTableSlots or not.
e. At executor startup determine if the entire plan tree supports
batch TupleTableSlots, if so enable batch scan mode.
I'm wondering if we should instead have a whole new kind of node, a
TupleTableVector, say. Things that want to work one tuple at a time
can continue to do so with no additional overhead. Things that want
to return batches can do it via this new result type.
node types, which *may* not be all that difficult. I'm also assuming
that batch mode (in all cases apart from queries with LIMIT or
cursors) will always be faster than tuple-at-a-time, so requires no
costings from the planner.
I definitely agree that we need to consider cases with and without
LIMIT separately, but there's more than one way to get a LIMIT. For
example, a subquery returns only one row; a semijoin returns only one
row. I don't think you are going to escape planner considerations.
Nested Loop Semi Join
-> Seq Scan
-> Index Scan on dont_batch_here
2. Vector processing
(I admit that I've given this part much less thought so far, but
here's what I have in mind)This depends on batch execution, and is intended to allow the executor
to perform function calls to an entire batch at once, rather than
tuple-at-a-time. For example, let's take the following example;SELECT a+b FROM t;
here (as of now) we'd scan "t" one row at a time and perform a+b after
having deformed enough of the tuple to do that. We'd then go and get
another Tuple from the scan node and repeat until the scan gave us no
more Tuples.With batch execution we'd fetch multiple Tuples from the scan and we'd
then perform the call to say int4_pl() multiple times, which still
kinda sucks as it means calling int4_pl() possibly millions of times
(once per tuple). The vector mode here would require that we modify
pg_operator to add a vector function for each operator so that we can
call the function passing in an array of Datums and a length to have
SIMD operations perform the addition, so we'd call something like
int4_pl_vector() only once per batch of tuples allowing the CPU to
perform SIMD operations on those datum arrays. This could be done in
an incremental way as the code could just callback on the standard
function in cases where a vectorised version of it is not available.
Thought is needed here about when exactly this decision is made as the
user may not have permissions to execute the vector function, so it
can't simply be a run time check. These functions would simply return
another vector of the results. Aggregates could be given a vector
transition function, where something like COUNT(*)'s vector_transfn
would simply just current_count += vector_length;This project does appear to require that we bloat the code with 100's
of vector versions of each function. I'm not quite sure if there's a
better way to handle this. The problem is that the fmgr is pretty much
a barrier to SIMD operations, and this was the only idea that I've had
so far about breaking through that barrier. So further ideas here are
very welcome.
I don't have any at the moment, but I'm not keen on hundreds of new
vector functions that can all have bugs or behavior differences versus
the unvectorized versions of the same code. That's a substantial tax
on future development. I think it's important to understand what
sorts of queries we are targeting here. KaiGai's GPU-acceleration
stuff does great on queries with complex WHERE clauses, but most
people don't care not only because it's out-of-core but because who
actually looks for the records where (a + b) % c > (d + e) * f / g?
This seems like it has the same issue. If we can speed up common
queries people are actually likely to run, OK, that's interesting.
By the way, I think KaiGai's GPU-acceleration stuff points to another
pitfall here. There's other stuff somebody might legitimately want to
do that requires another copy of each function. For example, run-time
code generation likely needs that (a function to tell the code
generator what to generate for each of our functions), and
GPU-acceleration probably does, too. If fixing a bug in numeric_lt
requires changing not only the regular version and the vectorized
version but also the GPU-accelerated version and the codegen version,
Tom and Dean are going to kill us. And justifiably so! Granted,
nobody is proposing those other features in core right now, but
they're totally reasonable things to want to do.
I suspect the number of queries that are being hurt by fmgr overhead
is really large, and I think it would be nice to attack that problem
more directly. It's a bit hard to discuss what's worthwhile in the
abstract, without performance numbers, but when you vectorize, how
much is the benefit from using SIMD instructions and how much is the
benefit from just not going through the fmgr every time?
The idea here is that these 2 projects help pave the way to bring
columnar storage into PostgreSQL. Without these we're unlikely to get
much benefit of columnar storage as we'd be stuck processing rows one
at a time still. Adding columnar storage on the top of the above
should further increase performance as we can skip the tuple-deform
step and pull columnar array/vectors directly into a TupleTableSlot,
although some trickery would be involved here when the scan has keys.
I'm a bit mystified by this. It seems to me that you could push down
the optimizable quals into the AM, just like what index AMs due for
Index Quals and what postgres_fdw does for pushdown-safe quals. Then
those quals get executed on the optimized representation, and you only
have to fill TupleTableSlots for the surviving tuples. AFAICS,
vectorizing the core executor only helps if you want to keep the data
in vectorized form for longer, e.g. to somehow optimize joins or aggs,
or if the data starts out in row-oriented form and we convert it to
columnar form before doing vector ops. Evidently I'm confused.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, May 9, 2016 at 9:38 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
Is the parallel aware Append node sufficient to run multiple nodes
asynchronously? (Sorry, I couldn't have enough time to code the feature
even though we had discussion before.)
It's tempting to think that parallel query and asynchronous query are
the same thing, but I think that they are actually quite different.
Parallel query involves using multiple processes to service a query.
Asynchronous query involves using each individual process as
efficiently as possible by not having it block any more than
necessary. You can want these things together or separately. For
example, consider this query plan:
Append
-> ForeignScan
-> ForeignScan
Here, you do not want parallel query; the queries must both be
launched by the user backend, not some worker process, else you will
not get the right transaction semantics. The parallel-aware Append
node we talked about before does not help here. On the other hand,
consider this:
Append
-> Seq Scan
Filter: lots_of_cpu()
-> Seq Scan
Filter: lots_of_cpu()
Here, asynchronous query is of no help, but parallel query is great.
Now consider this hypothetical plan:
Gather
-> Hash Join
-> Parallel Bitmap Heap Scan
-> Bitmap Index Scan
-> Parallel Hash
-> Parallel Seq Scan
Let's assume that the bitmap *heap* scan can be performed in parallel
but the bitmap *index* scan can't. That's pretty reasonable for a
first cut, actually, because the index accesses are probably touching
much less data, and we're doing little CPU work for each index page -
any delay here is likely to be I/O.
So in that world what you want is for the first worker to begin
performing the bitmap index scan and building a shared TIDBitmap for
that the workers can use to scan the heap. The other workers,
meanwhile, could begin building the shared hash table (which is what I
intend to denote by saying that it's a *Parallel* Hash). If the
process building the bitmap finishes before the hash table is built,
it can help build the hash table as well. Once both operations are
done, each process can scan a subset of the rows from the bitmap heap
scan and do the hash table probes for just those rows. To make all of
this work, you need both *parallel* query, so that you have workers,
and also *asynchronous* query, so that workers which see that the
bitmap index scan is in progress don't get stuck waiting for it but
can look around for other work.
In the above example, scan on foreign-table takes longer lead time than
local scan. If Append can map every child nodes on individual workers,
local scan worker begins to return tuples at first, then, mixed tuples
shall be returned eventually.
This is getting at an issue I'm somewhat worried about, which is
scheduling. In my prototype, we only check for events if there are no
nodes ready for the CPU now, but sometimes that might be a loser. One
probably needs to check for events periodically even when there are
still nodes waiting for the CPU, but "how often?" seems like an
unanswerable question.
However, the process internal asynchronous execution may be also beneficial
in case when cost of shm_mq is not ignorable (e.g, no scan qualifiers
are given to worker process). I think it allows to implement pre-fetching
very naturally.
Yes.
My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple.Backward compatibility is good. In addition, child node may want to
know the context when it is called. It may want to switch the behavior
according to the caller's expectation. For example, it may be beneficial
if SeqScan makes more aggressive prefetching on asynchronous execution.
Maybe, but I'm a bit doubtful. I'm not seeing a lot of advantage in
that sort of thing, and it will make the code a lot more complicated.
Also, can we consider which data format will be returned from the child
node during the planning stage? It affects to the cost of inter-node
data exchange. If a pair of parent-node and child-node supports its
special data format (like as existing HashJoin and Hash doing), it shall
be a discount factor of cost estimation.
I'm not sure. The costing aspects of this need a lot more thought
than I have given them so far.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, May 10, 2016 at 3:00 AM, konstantin knizhnik
<k.knizhnik@postgrespro.ru> wrote:
What's wrong with it that worker is blocked? You can just have more workers
(more than CPU cores) to let other of them continue to do useful work.
Not really. The workers are all running the same plan, so they'll all
make the same decision about which node needs to be executed next. If
that node can't accommodate multiple processes trying to execute it at
the same time, it will have to block all of them but the first one.
Adding more processes just increases the number of processes sitting
around doing nothing.
But there are some researches, for example:
http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
showing that the same or even better effect can be achieved by generation
native code for query execution plan (which is not so difficult now, thanks
to LLVM).
It eliminates interpretation overhead and increase cache locality.
I get similar results with my own experiments of accelerating SparkSQL.
Instead of native code generation I used conversion of query plans to C code
and experiment with different data representation. "Horisontal model" with
loading columns on demands shows better performance than columnar store.
Yes, I think this approach should also be considered.
At this moment (February) them have implemented translation of only few
PostgreSQL operators used by ExecQuals and do not support aggregates.
Them get about 2 times increase of speed at synthetic queries and 25%
increase at TPC-H Q1 (for Q1 most critical is generation of native code for
aggregates, because ExecQual itself takes only 6% of time for this query).
Actually these 25% for Q1 were achieved not by using dynamic code
generation, but switching from PULL to PUSH model in executor.
It seems to be yet another interesting PostgreSQL executor transformation.
As far as I know, them are going to publish result of their work to open
source...
Interesting. You may notice that in "asynchronous mode" my prototype
works using a push model of sorts. Maybe that should be taken
further.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 05/10/2016 08:26 PM, Robert Haas wrote:
On Tue, May 10, 2016 at 3:00 AM, konstantin knizhnik
<k.knizhnik@postgrespro.ru> wrote:What's wrong with it that worker is blocked? You can just have more workers
(more than CPU cores) to let other of them continue to do useful work.Not really. The workers are all running the same plan, so they'll all
make the same decision about which node needs to be executed next. If
that node can't accommodate multiple processes trying to execute it at
the same time, it will have to block all of them but the first one.
Adding more processes just increases the number of processes sitting
around doing nothing.
Doesn't this actually mean that we need to have normal job scheduler which is given queue of jobs and having some pool of threads will be able to orginize efficient execution of queries? Optimizer can build pipeline (graph) of tasks, which corresponds to
execution plan nodes, i.e. SeqScan, Sort, ... Each task is splitted into several jobs which can be concurretly scheduled by task dispatcher. So you will not have blocked worker waiting for something and all system resources will be utilized. Such approach
with dispatcher allows to implement quotas, priorities,... Also dispatches can care about NUMA and cache optimizations which is especially critical on modern architectures. One more reference: http://db.in.tum.de/~leis/papers/morsels.pdf
Sorry, may be I wrong, but I still think that async.ops is "multitasking for poor":)
Yes, maintaining threads and especially separate processes adds significant overhead. It leads to extra resources consumption and context switches are quite expensive. And I know from my own experience that replacing several concurrent processes performing
some IO (for example with sockets) with just one process using multiplexing allows to increase performance. But still async. ops. is just a way to make programmer responsible for managing state machine instead of relying on OS tomake context switches.
Manual transmission is still more efficient than automatic transmission. But still most drives prefer last one;)
Seriously, I carefully read your response to Kochei, but still not convinced that async. ops. is what we need. Or may be we just understand different thing by this notion.
But there are some researches, for example:
http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
showing that the same or even better effect can be achieved by generation
native code for query execution plan (which is not so difficult now, thanks
to LLVM).
It eliminates interpretation overhead and increase cache locality.
I get similar results with my own experiments of accelerating SparkSQL.
Instead of native code generation I used conversion of query plans to C code
and experiment with different data representation. "Horisontal model" with
loading columns on demands shows better performance than columnar store.Yes, I think this approach should also be considered.
At this moment (February) them have implemented translation of only few
PostgreSQL operators used by ExecQuals and do not support aggregates.
Them get about 2 times increase of speed at synthetic queries and 25%
increase at TPC-H Q1 (for Q1 most critical is generation of native code for
aggregates, because ExecQual itself takes only 6% of time for this query).
Actually these 25% for Q1 were achieved not by using dynamic code
generation, but switching from PULL to PUSH model in executor.
It seems to be yet another interesting PostgreSQL executor transformation.
As far as I know, them are going to publish result of their work to open
source...Interesting. You may notice that in "asynchronous mode" my prototype
works using a push model of sorts. Maybe that should be taken
further.
--
Konstantin Knizhnik
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 5/10/16 12:47 AM, Kouhei Kaigai wrote:
On 10 May 2016 at 13:38, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
My concern about ExecProcNode is, it is constructed with a large switch
... case statement. It involves tons of comparison operation at run-time.
If we replace this switch ... case by function pointer, probably, it make
performance improvement. Especially, OLAP workloads that process large
amount of rows.I imagined that any decent compiler would have built the code to use
jump tables for this. I have to say that I've never checked to make
sure though.Ah, indeed, you are right. Please forget above part.
Even so, I would think that the simplification in the executor would be
worth it. If you need to add a new node there's dozens of places where
you might have to mess with these giant case statements.
In python (for example), types (equivalent to nodes in this case) have
data structures that define function pointers for a core set of
operations (such as doing garbage collection, or generating a string
representation). That means that to add a new type at the C level you
only need to define a C structure that has the expected members, and an
initializer function that will properly set everything up when you
create a new instance. There's no messing around with the rest of the
guts of python.
*Even more important, everything you need to know about the type is
contained in one place, not spread throughout the code.*
--
Jim Nasby, Data Architect, Blue Treble Consulting, Austin TX
Experts in Analytics, Data Architecture and PostgreSQL
Data in Trouble? Get it in Treble! http://BlueTreble.com
855-TREBLE2 (855-873-2532) mobile: 512-569-9461
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
hmm, the morsels paper looks really interesting at first sight.
Let's see if we can get a poc working in PostgreSQL? :-)
On Tue, May 10, 2016 at 9:42 PM, Konstantin Knizhnik <
k.knizhnik@postgrespro.ru> wrote:
On 05/10/2016 08:26 PM, Robert Haas wrote:
On Tue, May 10, 2016 at 3:00 AM, konstantin knizhnik
<k.knizhnik@postgrespro.ru> wrote:What's wrong with it that worker is blocked? You can just have more
workers
(more than CPU cores) to let other of them continue to do useful work.Not really. The workers are all running the same plan, so they'll all
make the same decision about which node needs to be executed next. If
that node can't accommodate multiple processes trying to execute it at
the same time, it will have to block all of them but the first one.
Adding more processes just increases the number of processes sitting
around doing nothing.Doesn't this actually mean that we need to have normal job scheduler which
is given queue of jobs and having some pool of threads will be able to
orginize efficient execution of queries? Optimizer can build pipeline
(graph) of tasks, which corresponds to execution plan nodes, i.e. SeqScan,
Sort, ... Each task is splitted into several jobs which can be concurretly
scheduled by task dispatcher. So you will not have blocked worker waiting
for something and all system resources will be utilized. Such approach with
dispatcher allows to implement quotas, priorities,... Also dispatches can
care about NUMA and cache optimizations which is especially critical on
modern architectures. One more reference:
http://db.in.tum.de/~leis/papers/morsels.pdfSorry, may be I wrong, but I still think that async.ops is "multitasking
for poor":)
Yes, maintaining threads and especially separate processes adds
significant overhead. It leads to extra resources consumption and context
switches are quite expensive. And I know from my own experience that
replacing several concurrent processes performing some IO (for example with
sockets) with just one process using multiplexing allows to increase
performance. But still async. ops. is just a way to make programmer
responsible for managing state machine instead of relying on OS tomake
context switches. Manual transmission is still more efficient than
automatic transmission. But still most drives prefer last one;)Seriously, I carefully read your response to Kochei, but still not
convinced that async. ops. is what we need. Or may be we just understand
different thing by this notion.But there are some researches, for example:
http://www.vldb.org/pvldb/vol4/p539-neumann.pdf
showing that the same or even better effect can be achieved by generation
native code for query execution plan (which is not so difficult now,
thanks
to LLVM).
It eliminates interpretation overhead and increase cache locality.
I get similar results with my own experiments of accelerating SparkSQL.
Instead of native code generation I used conversion of query plans to C
code
and experiment with different data representation. "Horisontal model"
with
loading columns on demands shows better performance than columnar store.Yes, I think this approach should also be considered.
At this moment (February) them have implemented translation of only few
PostgreSQL operators used by ExecQuals and do not support aggregates.
Them get about 2 times increase of speed at synthetic queries and 25%
increase at TPC-H Q1 (for Q1 most critical is generation of native code
for
aggregates, because ExecQual itself takes only 6% of time for this
query).
Actually these 25% for Q1 were achieved not by using dynamic code
generation, but switching from PULL to PUSH model in executor.
It seems to be yet another interesting PostgreSQL executor
transformation.
As far as I know, them are going to publish result of their work to open
source...Interesting. You may notice that in "asynchronous mode" my prototype
works using a push model of sorts. Maybe that should be taken
further.--
Konstantin Knizhnik
Postgres Professional: http://www.postgrespro.com
The Russian Postgres Company--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
--
Bert Desmet
0477/305361
Hi,
On 2016-05-09 13:33:55 -0400, Robert Haas wrote:
I think there are several different areas
where we should consider major upgrades to our executor. It's too
slow and it doesn't do everything we want it to do. The main things
on my mind are:
3) We use a lot of very cache-inefficient datastructures.
Especially the pervasive use of linked lists in the executor is pretty
bad for performance. Every element is likely to incur cache misses,
every list element pretty much has it's own cacheline (thereby reducing
overall cache hit ratio), they have a horrible allocation overhead (both
space and palloc runtime).
1. asynchronous execution, by which I mean the ability of a node to
somehow say that it will generate a tuple eventually, but is not yet
ready, so that the executor can go run some other part of the plan
tree while it waits. [...]. It is also a problem
for parallel query: in a parallel sequential scan, the next worker can
begin reading the next block even if the current block hasn't yet been
received from the OS. Whether or not this will be efficient is a
research question, but it can be done. However, imagine a parallel
scan of a btree index: we don't know what page to scan next until we
read the previous page and examine the next-pointer. In the meantime,
any worker that arrives at that scan node has no choice but to block.
It would be better if the scan node could instead say "hey, thanks for
coming but I'm really not ready to be on-CPU just at the moment" and
potentially allow the worker to go work in some other part of the
query tree. For that worker to actually find useful work to do
elsewhere, we'll probably need it to be the case either that the table
is partitioned or the original query will need to involve UNION ALL,
but those are not silly cases to worry about, particularly if we get
native partitioning in 9.7.
I've to admit I'm not that convinced about the speedups in the !fdw
case. There seems to be a lot easier avenues for performance
improvements.
2. vectorized execution, by which I mean the ability of a node to
return tuples in batches rather than one by one. Andres has opined
more than once that repeated trips through ExecProcNode defeat the
ability of the CPU to do branch prediction correctly, slowing the
whole system down, and that they also result in poor CPU cache
behavior, since we jump all over the place executing a little bit of
code from each node before moving onto the next rather than running
one bit of code first, and then another later.
FWIW, I've even hacked something up for a bunch of simple queries, and
the performance improvements were significant. Besides it only being a
weekend hack project, the big thing I got stuck on was considering how
to exactly determine when to batch and not to batch.
I'd personally say that the CPU pipeline defeating aspect is worse than
the effect of the cache/branch misses themselves. Today's CPUs are
heavily superscalar, and our instruction-per-cycle throughput shows
pretty clearly that we're not good at employing (micro-)instruction
paralellism. We're quite frequently at well below one instruction/cycle.
My proposal for how to do this is to make ExecProcNode function as a
backward-compatibility wrapper. For asynchronous execution, a node
might return a not-ready-yet indication, but if that node is called
via ExecProcNode, it means the caller isn't prepared to receive such
an indication, so ExecProcNode will just wait for the node to become
ready and then return the tuple. Similarly, for vectorized execution,
a node might return a bunch of tuples all at once. ExecProcNode will
extract the first one and return it to the caller, and subsequent
calls to ExecProcNode will iterate through the rest of the batch, only
calling the underlying node-specific function when the batch is
exhausted. In this way, code that doesn't know about the new stuff
can continue to work pretty much as it does today.
I agree that that generally is a reasonable way forward.
Also, and I think
this is important, nodes don't need the permission of their parent
node to use these new capabilities. They can use them whenever they
wish, without worrying about whether the upper node is prepared to
deal with it. If not, ExecProcNode will paper over the problem. This
seems to me to be a good way to keep the code simple.
Maybe not permission, but for some cases it seems to be important to
hint to *not* prefetch a lot of rows. E.g. anti joins come to mind. Just
using batching with force seems likely to regress some queries quite
badly (e.g an expensive join inside an EXISTS() which returns many
tuples).
For asynchronous execution, I have gone so far as to mock up a bit of
what this might look like. This shouldn't be taken very seriously at
this point, but I'm attaching a few very-much-WIP patches to show the
direction of my line of thinking. Basically, I propose to have
ExecBlah (that is, ExecBitmapHeapScan, ExecAppend, etc.) return tuples
by putting them into a new PlanState member called "result", which is
just a Node * so that we can support multiple types of results,
instead of returning them.
What different types of results are you envisioning?
By the way, one smaller executor project that I think we should also
look at has to do with this comment in nodeSeqScan.c:static bool
SeqRecheck(SeqScanState *node, TupleTableSlot *slot)
{
/*
* Note that unlike IndexScan, SeqScan never use keys in heap_beginscan
* (and this is very bad) - so, here we do not check are keys ok or not.
*/
return true;
}Some quick prototyping by my colleague Dilip Kumar suggests that, in
fact, there are cases where pushing down keys into heap_beginscan()
could be significantly faster.
I can immediately believe that.
Some care is required here because any
functions we execute as scan keys are run with the buffer locked, so
we had better not run anything very complicated. But doing this for
simple things like integer equality operators seems like it could save
quite a few buffer lock/unlock cycles and some other executor overhead
as well.
Hm. Do we really have to keep the page locked in the page-at-a-time
mode? Shouldn't the pin suffice?
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, May 10, 2016 at 7:56 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Mon, May 9, 2016 at 8:34 PM, David Rowley
<david.rowley@2ndquadrant.com> wrote:
I don't have any at the moment, but I'm not keen on hundreds of new
vector functions that can all have bugs or behavior differences versus
the unvectorized versions of the same code. That's a substantial tax
on future development. I think it's important to understand what
sorts of queries we are targeting here. KaiGai's GPU-acceleration
stuff does great on queries with complex WHERE clauses, but most
people don't care not only because it's out-of-core but because who
actually looks for the records where (a + b) % c > (d + e) * f / g?
This seems like it has the same issue. If we can speed up common
queries people are actually likely to run, OK, that's interesting.
I have seen pretty complex expressions in the projection and
aggregation. Couple dozen SUM(CASE WHEN a THEN b*c ELSE MIN(d,e)*f
END) type of expressions. In critical places had to replace them with
a C coded function that processed a row at a time to avoid the
executor dispatch overhead.
By the way, I think KaiGai's GPU-acceleration stuff points to another
pitfall here. There's other stuff somebody might legitimately want to
do that requires another copy of each function. For example, run-time
code generation likely needs that (a function to tell the code
generator what to generate for each of our functions), and
GPU-acceleration probably does, too. If fixing a bug in numeric_lt
requires changing not only the regular version and the vectorized
version but also the GPU-accelerated version and the codegen version,
Tom and Dean are going to kill us. And justifiably so! Granted,
nobody is proposing those other features in core right now, but
they're totally reasonable things to want to do.
My thoughts in this area have been circling around getting LLVM to do
the heavy lifting. LLVM/clang could compile existing C functions to IR
and bundle those with the DB. At query planning time or maybe even
during execution the functions can be inlined into the compiled query
plan, LLVM can then be coaxed to copy propagate, constant fold and
dead code eliminate the bejeezus out of the expression tree. This way
duplication of the specialized code can be kept to a minimum while at
least the common cases can completely avoid the fmgr overhead.
This approach would also mesh together fine with batching. Given
suitably regular data structures and simple functions, LLVM will be
able to vectorize the code. If not it will still end up with a nice
tight loop that is an order of magnitude or two faster than the
current executor.
The first cut could take care of ExecQual, ExecTargetList and friends.
Later improvements could let execution nodes provide basic blocks that
would then be threaded together into the main execution loop. If some
node does not implement the basic block interface a default
implementation is used that calls the current interface. It gets a bit
handwavy at this point, but the main idea would be to enable data
marshaling so that values can be routed directly to the code that
needs them without being written to intermediate storage.
I suspect the number of queries that are being hurt by fmgr overhead
is really large, and I think it would be nice to attack that problem
more directly. It's a bit hard to discuss what's worthwhile in the
abstract, without performance numbers, but when you vectorize, how
much is the benefit from using SIMD instructions and how much is the
benefit from just not going through the fmgr every time?
My feeling is the same, fmgr overhead and data marshaling, dynamic
dispatch through the executor is the big issue. This is corroborated
by what I have seen found by other VM implementations. Once you get
the data into an uniform format where vectorized execution could be
used, the CPU execution resources are no longer the bottleneck. Memory
bandwidth gets in the way, unless each input value is used in multiple
calculations. And even then, we are looking at a 4x speedup at best.
Regards,
Ants Aasma
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Import Notes
Reply to msg id not found: 1869930871.370126.1462899392851@RIA