Thread: Parallel Seq Scan
Attachment
On 12/04/2014 07:35 AM, Amit Kapila wrote: > [snip] > > The number of worker backends that can be used for > parallel seq scan can be configured by using a new GUC > parallel_seqscan_degree, the default value of which is zero > and it means parallel seq scan will not be considered unless > user configures this value. The number of parallel workers should be capped (of course!) at the maximum amount of "processors" (cores/vCores, threads/hyperthreads) available. More over, when load goes up, the relative cost of parallel working should go up as well. Something like: p = number of cores l = 1min-load additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1) (for c>1, of course) > In ExecutorStart phase, initiate the required number of workers > as per parallel seq scan plan and setup dynamic shared memory and > share the information required for worker to execute the scan. > Currently I have just shared the relId, targetlist and number > of blocks to be scanned by worker, however I think we might want > to generate a plan for each of the workers in master backend and > then share the same to individual worker. [snip] > Attached patch is just to facilitate the discussion about the > parallel seq scan and may be some other dependent tasks like > sharing of various states like combocid, snapshot with parallel > workers. It is by no means ready to do any complex test, ofcourse > I will work towards making it more robust both in terms of adding > more stuff and doing performance optimizations. > > Thoughts/Suggestions? Not directly (I haven't had the time to read the code yet), but I'm thinking about the ability to simply *replace* executor methods from an extension. This could be an alternative to providing additional nodes that the planner can include in the final plan tree, ready to be executed. The parallel seq scan nodes are definitively the best approach for "parallel query", since the planner can optimize them based on cost. I'm wondering about the ability to modify the implementation of some methods themselves once at execution time: given a previously planned query, chances are that, at execution time (I'm specifically thinking about prepared statements here), a different implementation of the same "node" might be more suitable and could be used instead while the condition holds. If this latter line of thinking is too off-topic within this thread and there is any interest, we can move the comments to another thread and I'd begin work on a PoC patch. It might as well make sense to implement the executor overloading mechanism alongide the custom plan API, though. Any comments appreciated. Thank you for your work, Amit Regards, / J.L.
José, * José Luis Tallón (jltallon@adv-solutions.net) wrote: > On 12/04/2014 07:35 AM, Amit Kapila wrote: > >The number of worker backends that can be used for > >parallel seq scan can be configured by using a new GUC > >parallel_seqscan_degree, the default value of which is zero > >and it means parallel seq scan will not be considered unless > >user configures this value. > > The number of parallel workers should be capped (of course!) at the > maximum amount of "processors" (cores/vCores, threads/hyperthreads) > available. > > More over, when load goes up, the relative cost of parallel working > should go up as well. > Something like: > p = number of cores > l = 1min-load > > additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1) > > (for c>1, of course) While I agree in general that we'll need to come up with appropriate acceptance criteria, etc, I don't think we want to complicate this patch with that initially. A SUSET GUC which caps the parallel GUC would be enough for an initial implementation, imv. > Not directly (I haven't had the time to read the code yet), but I'm > thinking about the ability to simply *replace* executor methods from > an extension. You probably want to look at the CustomScan thread+patch directly then.. Thanks, Stephen
Amit, * Amit Kapila (amit.kapila16@gmail.com) wrote: > postgres=# explain select c1 from t1; > QUERY PLAN > ------------------------------------------------------ > Seq Scan on t1 (cost=0.00..101.00 rows=100 width=4) > (1 row) > > > postgres=# set parallel_seqscan_degree=4; > SET > postgres=# explain select c1 from t1; > QUERY PLAN > -------------------------------------------------------------- > Parallel Seq Scan on t1 (cost=0.00..25.25 rows=100 width=4) > Number of Workers: 4 > Number of Blocks Per Workers: 25 > (3 rows) This is all great and interesting, but I feel like folks might be waiting to see just what kind of performance results come from this (and what kind of hardware is needed to see gains..). There's likely to be situations where this change is an improvement while also being cases where it makes things worse. One really interesting case would be parallel seq scans which are executing against foreign tables/FDWs.. Thanks! Stephen
On 12/5/14, 9:08 AM, José Luis Tallón wrote: > > More over, when load goes up, the relative cost of parallel working should go up as well. > Something like: > p = number of cores > l = 1min-load > > additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1) > > (for c>1, of course) ... > The parallel seq scan nodes are definitively the best approach for "parallel query", since the planner can optimize thembased on cost. > I'm wondering about the ability to modify the implementation of some methods themselves once at execution time: given apreviously planned query, chances are that, at execution time (I'm specifically thinking about prepared statements here),a different implementation of the same "node" might be more suitable and could be used instead while the conditionholds. These comments got me wondering... would it be better to decide on parallelism during execution instead of at plan time?That would allow us to dynamically scale parallelism based on system load. If we don't even consider parallelism untilwe've pulled some number of tuples/pages from a relation, this would also eliminate all parallel overhead on small relations. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
>
> On 12/04/2014 07:35 AM, Amit Kapila wrote:
>>
>> [snip]
>>
>> The number of worker backends that can be used for
>> parallel seq scan can be configured by using a new GUC
>> parallel_seqscan_degree, the default value of which is zero
>> and it means parallel seq scan will not be considered unless
>> user configures this value.
>
>
> The number of parallel workers should be capped (of course!) at the maximum amount of "processors" (cores/vCores, threads/hyperthreads) available.
>
> More over, when load goes up, the relative cost of parallel working should go up as well.
> Something like:
> p = number of cores
> l = 1min-load
>
> additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1)
>
> (for c>1, of course)
>
>
>> In ExecutorStart phase, initiate the required number of workers
>> as per parallel seq scan plan and setup dynamic shared memory and
>> share the information required for worker to execute the scan.
>> Currently I have just shared the relId, targetlist and number
>> of blocks to be scanned by worker, however I think we might want
>> to generate a plan for each of the workers in master backend and
>> then share the same to individual worker.
>
> [snip]
>>
>> Attached patch is just to facilitate the discussion about the
>> parallel seq scan and may be some other dependent tasks like
>> sharing of various states like combocid, snapshot with parallel
>> workers. It is by no means ready to do any complex test, ofcourse
>> I will work towards making it more robust both in terms of adding
>> more stuff and doing performance optimizations.
>>
>> Thoughts/Suggestions?
>
>
> Not directly (I haven't had the time to read the code yet), but I'm thinking about the ability to simply *replace* executor methods from an extension.
> This could be an alternative to providing additional nodes that the planner can include in the final plan tree, ready to be executed.
>
> The parallel seq scan nodes are definitively the best approach for "parallel query", since the planner can optimize them based on cost.
> I'm wondering about the ability to modify the implementation of some methods themselves once at execution time: given a previously planned query, chances are that, at execution time (I'm specifically thinking about prepared statements here), a different implementation of the same "node" might be more suitable and could be used instead while the condition holds.
>
> If this latter line of thinking is too off-topic within this thread and there is any interest, we can move the comments to another thread and I'd begin work on a PoC patch. It might as well make sense to implement the executor overloading mechanism alongide the custom plan API, though.
> Any comments appreciated.
>
>
> Thank you for your work, Amit
Attached patch is just to facilitate the discussion about theparallel seq scan and may be some other dependent tasks likesharing of various states like combocid, snapshot with parallelworkers. It is by no means ready to do any complex test, ofcourseI will work towards making it more robust both in terms of addingmore stuff and doing performance optimizations.Thoughts/Suggestions?
Attachment
>
> Amit,
>
> * Amit Kapila (amit.kapila16@gmail.com) wrote:
> > postgres=# explain select c1 from t1;
> > QUERY PLAN
> > ------------------------------------------------------
> > Seq Scan on t1 (cost=0.00..101.00 rows=100 width=4)
> > (1 row)
> >
> >
> > postgres=# set parallel_seqscan_degree=4;
> > SET
> > postgres=# explain select c1 from t1;
> > QUERY PLAN
> > --------------------------------------------------------------
> > Parallel Seq Scan on t1 (cost=0.00..25.25 rows=100 width=4)
> > Number of Workers: 4
> > Number of Blocks Per Workers: 25
> > (3 rows)
>
> This is all great and interesting, but I feel like folks might be
> waiting to see just what kind of performance results come from this (and
> what kind of hardware is needed to see gains..).
> situations where this change is an improvement while also being cases
> where it makes things worse.
Agreed and I think that will be more clear after doing some
> One really interesting case would be parallel seq scans which are
> executing against foreign tables/FDWs..
>
>
> José,
>
> * José Luis Tallón (jltallon@adv-solutions.net) wrote:
> > On 12/04/2014 07:35 AM, Amit Kapila wrote:
> > >The number of worker backends that can be used for
> > >parallel seq scan can be configured by using a new GUC
> > >parallel_seqscan_degree, the default value of which is zero
> > >and it means parallel seq scan will not be considered unless
> > >user configures this value.
> >
> > The number of parallel workers should be capped (of course!) at the
> > maximum amount of "processors" (cores/vCores, threads/hyperthreads)
> > available.
> >
> > More over, when load goes up, the relative cost of parallel working
> > should go up as well.
> > Something like:
> > p = number of cores
> > l = 1min-load
> >
> > additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1)
> >
> > (for c>1, of course)
>
> While I agree in general that we'll need to come up with appropriate
> acceptance criteria, etc, I don't think we want to complicate this patch
> with that initially.
> enough for an initial implementation, imv.
>
This is exactly what I have done in patch.
> On 12/5/14, 9:08 AM, José Luis Tallón wrote:
>>
>>
>> More over, when load goes up, the relative cost of parallel working should go up as well.
>> Something like:
>> p = number of cores
>> l = 1min-load
>>
>> additional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1)
>>
>> (for c>1, of course)
>
>
> ...
>
>> The parallel seq scan nodes are definitively the best approach for "parallel query", since the planner can optimize them based on cost.
>> I'm wondering about the ability to modify the implementation of some methods themselves once at execution time: given a previously planned query, chances are that, at execution time (I'm specifically thinking about prepared statements here), a different implementation of the same "node" might be more suitable and could be used instead while the condition holds.
>
>
> These comments got me wondering... would it be better to decide on parallelism during execution instead of at plan time? That would allow us to dynamically scale parallelism based on system load. If we don't even consider parallelism until we've pulled some number of tuples/pages from a relation,
> --
I think we have access to this information in planner (RelOptInfo -> pages),
> On 4 December 2014 at 19:35, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> Attached patch is just to facilitate the discussion about the
>> parallel seq scan and may be some other dependent tasks like
>> sharing of various states like combocid, snapshot with parallel
>> workers. It is by no means ready to do any complex test, ofcourse
>> I will work towards making it more robust both in terms of adding
>> more stuff and doing performance optimizations.
>>
>> Thoughts/Suggestions?
>>
>
> This is good news!
> I've not gotten to look at the patch yet, but I thought you may be able to make use of the attached at some point.
>
> It's bare-bones core support for allowing aggregate states to be merged together with another aggregate state. I would imagine that if a query such as:
>
> SELECT MAX(value) FROM bigtable;
>
> was run, then a series of parallel workers could go off and each find the max value from their portion of the table and then perhaps some other node type would then take all the intermediate results from the workers, once they're finished, and join all of the aggregate states into one and return that. Naturally, you'd need to check that all aggregates used in the targetlist had a merge function first.
>
> This is just a few hours of work. I've not really tested the pg_dump support or anything yet. I've also not added any new functions to allow AVG() or COUNT() to work, I've really just re-used existing functions where I could, as things like MAX() and BOOL_OR() can just make use of the existing transition function. I thought that this might be enough for early tests.
>
> I'd imagine such a workload, ignoring IO overhead, should scale pretty much linearly with the number of worker processes. Of course, if there was a GROUP BY clause then the merger code would have to perform more work.
>
> If you think you might be able to make use of this, then I'm willing to go off and write all the other merge functions required for the other aggregates.
>
* Amit Kapila (amit.kapila16@gmail.com) wrote: > 1. As the patch currently stands, it just shares the relevant > data (like relid, target list, block range each worker should > perform on etc.) to the worker and then worker receives that > data and form the planned statement which it will execute and > send the results back to master backend. So the question > here is do you think it is reasonable or should we try to form > the complete plan for each worker and then share the same > and may be other information as well like range table entries > which are required. My personal gut feeling in this matter > is that for long term it might be better to form the complete > plan of each worker in master and share the same, however > I think the current way as done in patch (okay that needs > some improvement) is also not bad and quite easier to implement. For my 2c, I'd like to see it support exactly what the SeqScan node supports and then also what Foreign Scan supports. That would mean we'd then be able to push filtering down to the workers which would be great. Even better would be figuring out how to parallelize an Append node (perhaps only possible when the nodes underneath are all SeqScan or ForeignScan nodes) since that would allow us to then parallelize the work across multiple tables and remote servers. One of the big reasons why I was asking about performance data is that, today, we can't easily split a single relation across multiple i/o channels. Sure, we can use RAID and get the i/o channel that the table sits on faster than a single disk and possibly fast enough that a single CPU can't keep up, but that's not quite the same. The historical recommendations for Hadoop nodes is around one CPU per drive (of course, it'll depend on workload, etc, etc, but still) and while there's still a lot of testing, etc, to be done before we can be sure about the 'right' answer for PG (and it'll also vary based on workload, etc), that strikes me as a pretty reasonable rule-of-thumb to go on. Of course, I'm aware that this won't be as easy to implement.. > 2. Next question related to above is what should be the > output of ExplainPlan, as currently worker is responsible > for forming its own plan, Explain Plan is not able to show > the detailed plan for each worker, is that okay? I'm not entirely following this. How can the worker be responsible for its own "plan" when the information passed to it (per the above paragraph..) is pretty minimal? In general, I don't think we need to have specifics like "this worker is going to do exactly X" because we will eventually need some communication to happen between the worker and the master process where the worker can ask for more work because it's finished what it was tasked with and the master will need to give it another chunk of work to do. I don't think we want exactly what each worker process will do to be fully formed at the outset because, even with the best information available, given concurrent load on the system, it's not going to be perfect and we'll end up starving workers. The plan, as formed by the master, should be more along the lines of "this is what I'm gonna have my workers do" along w/ how many workers, etc, and then it goes and does it. Perhaps for an 'explain analyze' we return information about what workers actually *did* what, but that's a whole different discussion. > 3. Some places where optimizations are possible: > - Currently after getting the tuple from heap, it is deformed by > worker and sent via message queue to master backend, master > backend then forms the tuple and send it to upper layer which > before sending it to frontend again deforms it via slot_getallattrs(slot). If this is done as I was proposing above, we might be able to avoid this, but I don't know that it's a huge issue either way.. The bigger issue is getting the filtering pushed down. > - Master backend currently receives the data from multiple workers > serially. We can optimize in a way that it can check other queues, > if there is no data in current queue. Yes, this is pretty critical. In fact, it's one of the recommendations I made previously about how to change the Append node to parallelize Foreign Scan node work. > - Master backend is just responsible for coordination among workers > It shares the required information to workers and then fetch the > data processed by each worker, by using some more logic, we might > be able to make master backend also fetch data from heap rather than > doing just co-ordination among workers. I don't think this is really necessary... > I think in all above places we can do some optimisation, however > we can do that later as well, unless they hit the performance badly for > cases which people care most. I agree that we can improve the performance through various optimizations later, but it's important to get the general structure and design right or we'll end up having to reimplement a lot of it. > 4. Should parallel_seqscan_degree value be dependent on other > backend processes like MaxConnections, max_worker_processes, > autovacuum_max_workers do or should it be independent like > max_wal_senders? Well, we're not going to be able to spin off more workers than we have process slots, but I'm not sure we need anything more than that? In any case, this is definitely an area we can work on improving later and I don't think it really impacts the rest of the design. Thanks, Stephen
>
> * Amit Kapila (amit.kapila16@gmail.com) wrote:
> > 1. As the patch currently stands, it just shares the relevant
> > data (like relid, target list, block range each worker should
> > perform on etc.) to the worker and then worker receives that
> > data and form the planned statement which it will execute and
> > send the results back to master backend. So the question
> > here is do you think it is reasonable or should we try to form
> > the complete plan for each worker and then share the same
> > and may be other information as well like range table entries
> > which are required. My personal gut feeling in this matter
> > is that for long term it might be better to form the complete
> > plan of each worker in master and share the same, however
> > I think the current way as done in patch (okay that needs
> > some improvement) is also not bad and quite easier to implement.
>
> For my 2c, I'd like to see it support exactly what the SeqScan node
> supports and then also what Foreign Scan supports. That would mean we'd
> then be able to push filtering down to the workers which would be great.
> Even better would be figuring out how to parallelize an Append node
> (perhaps only possible when the nodes underneath are all SeqScan or
> ForeignScan nodes) since that would allow us to then parallelize the
> work across multiple tables and remote servers.
>
> One of the big reasons why I was asking about performance data is that,
> today, we can't easily split a single relation across multiple i/o
> channels. Sure, we can use RAID and get the i/o channel that the table
> sits on faster than a single disk and possibly fast enough that a single
> CPU can't keep up, but that's not quite the same. The historical
> recommendations for Hadoop nodes is around one CPU per drive (of course,
> it'll depend on workload, etc, etc, but still) and while there's still a
> lot of testing, etc, to be done before we can be sure about the 'right'
> answer for PG (and it'll also vary based on workload, etc), that strikes
> me as a pretty reasonable rule-of-thumb to go on.
>
> Of course, I'm aware that this won't be as easy to implement..
>
> > 2. Next question related to above is what should be the
> > output of ExplainPlan, as currently worker is responsible
> > for forming its own plan, Explain Plan is not able to show
> > the detailed plan for each worker, is that okay?
>
> I'm not entirely following this. How can the worker be responsible for
> its own "plan" when the information passed to it (per the above
> paragraph..) is pretty minimal?
> have specifics like "this worker is going to do exactly X" because we
> will eventually need some communication to happen between the worker and
> the master process where the worker can ask for more work because it's
> finished what it was tasked with and the master will need to give it
> another chunk of work to do. I don't think we want exactly what each
> worker process will do to be fully formed at the outset because, even
> with the best information available, given concurrent load on the
> system, it's not going to be perfect and we'll end up starving workers.
> The plan, as formed by the master, should be more along the lines of
> "this is what I'm gonna have my workers do" along w/ how many workers,
> etc, and then it goes and does it.
> return information about what workers actually *did* what, but that's a
> whole different discussion.
>
> > 3. Some places where optimizations are possible:
> > - Currently after getting the tuple from heap, it is deformed by
> > worker and sent via message queue to master backend, master
> > backend then forms the tuple and send it to upper layer which
> > before sending it to frontend again deforms it via slot_getallattrs(slot).
>
> If this is done as I was proposing above, we might be able to avoid
> this, but I don't know that it's a huge issue either way.. The bigger
> issue is getting the filtering pushed down.
>
> > - Master backend currently receives the data from multiple workers
> > serially. We can optimize in a way that it can check other queues,
> > if there is no data in current queue.
>
> Yes, this is pretty critical. In fact, it's one of the recommendations
> I made previously about how to change the Append node to parallelize
> Foreign Scan node work.
>
> > - Master backend is just responsible for coordination among workers
> > It shares the required information to workers and then fetch the
> > data processed by each worker, by using some more logic, we might
> > be able to make master backend also fetch data from heap rather than
> > doing just co-ordination among workers.
>
> I don't think this is really necessary...
>
> > I think in all above places we can do some optimisation, however
> > we can do that later as well, unless they hit the performance badly for
> > cases which people care most.
>
> I agree that we can improve the performance through various
> optimizations later, but it's important to get the general structure and
> design right or we'll end up having to reimplement a lot of it.
>
On Sat, Dec 6, 2014 at 12:13 AM, David Rowley <dgrowleyml@gmail.com> wrote: > It's bare-bones core support for allowing aggregate states to be merged > together with another aggregate state. I would imagine that if a query such > as: > > SELECT MAX(value) FROM bigtable; > > was run, then a series of parallel workers could go off and each find the > max value from their portion of the table and then perhaps some other node > type would then take all the intermediate results from the workers, once > they're finished, and join all of the aggregate states into one and return > that. Naturally, you'd need to check that all aggregates used in the > targetlist had a merge function first. I think this is great infrastructure and could also be useful for pushing down aggregates in cases involving foreign data wrappers. But I suggest we discuss it on a separate thread because it's not related to parallel seq scan per se. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Dec 6, 2014 at 1:50 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I think we have access to this information in planner (RelOptInfo -> pages), > if we want, we can use that to eliminate the small relations from > parallelism, but question is how big relations do we want to consider > for parallelism, one way is to check via tests which I am planning to > follow, do you think we have any heuristic which we can use to decide > how big relations should be consider for parallelism? Surely the Path machinery needs to decide this in particular cases based on cost. We should assign some cost to starting a parallel worker via some new GUC, like parallel_startup_cost = 100,000. And then we should also assign a cost to the act of relaying a tuple from the parallel worker to the master, maybe cpu_tuple_cost (or some new GUC). For a small relation, or a query with a LIMIT clause, the parallel startup cost will make starting a lot of workers look unattractive, but for bigger relations it will make sense from a cost perspective, which is exactly what we want. There are probably other important considerations based on goals for overall resource utilization, and also because at a certain point adding more workers won't help because the disk will be saturated. I don't know exactly what we should do about those issues yet, but the steps described in the previous paragraph seem like a good place to start anyway. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Dec 6, 2014 at 7:07 AM, Stephen Frost <sfrost@snowman.net> wrote: > For my 2c, I'd like to see it support exactly what the SeqScan node > supports and then also what Foreign Scan supports. That would mean we'd > then be able to push filtering down to the workers which would be great. > Even better would be figuring out how to parallelize an Append node > (perhaps only possible when the nodes underneath are all SeqScan or > ForeignScan nodes) since that would allow us to then parallelize the > work across multiple tables and remote servers. I don't see how we can support the stuff ForeignScan does; presumably any parallelism there is up to the FDW to implement, using whatever in-core tools we provide. I do agree that parallelizing Append nodes is useful; but let's get one thing done first before we start trying to do thing #2. > I'm not entirely following this. How can the worker be responsible for > its own "plan" when the information passed to it (per the above > paragraph..) is pretty minimal? In general, I don't think we need to > have specifics like "this worker is going to do exactly X" because we > will eventually need some communication to happen between the worker and > the master process where the worker can ask for more work because it's > finished what it was tasked with and the master will need to give it > another chunk of work to do. I don't think we want exactly what each > worker process will do to be fully formed at the outset because, even > with the best information available, given concurrent load on the > system, it's not going to be perfect and we'll end up starving workers. > The plan, as formed by the master, should be more along the lines of > "this is what I'm gonna have my workers do" along w/ how many workers, > etc, and then it goes and does it. Perhaps for an 'explain analyze' we > return information about what workers actually *did* what, but that's a > whole different discussion. I agree with this. For a first version, I think it's OK to start a worker up for a particular sequential scan and have it help with that sequential scan until the scan is completed, and then exit. It should not, as the present version of the patch does, assign a fixed block range to each worker; instead, workers should allocate a block or chunk of blocks to work on until no blocks remain. That way, even if every worker but one gets stuck, the rest of the scan can still finish. Eventually, we will want to be smarter about sharing works between multiple parts of the plan, but I think it is just fine to leave that as a future enhancement for now. >> - Master backend is just responsible for coordination among workers >> It shares the required information to workers and then fetch the >> data processed by each worker, by using some more logic, we might >> be able to make master backend also fetch data from heap rather than >> doing just co-ordination among workers. > > I don't think this is really necessary... I think it would be an awfully good idea to make this work. The master thread may be significantly faster than any of the others because it has no IPC costs. We don't want to leave our best resource sitting on the bench. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Sat, Dec 6, 2014 at 1:50 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I think we have access to this information in planner (RelOptInfo -> pages),
> > if we want, we can use that to eliminate the small relations from
> > parallelism, but question is how big relations do we want to consider
> > for parallelism, one way is to check via tests which I am planning to
> > follow, do you think we have any heuristic which we can use to decide
> > how big relations should be consider for parallelism?
>
> Surely the Path machinery needs to decide this in particular cases
> based on cost. We should assign some cost to starting a parallel
> worker via some new GUC, like parallel_startup_cost = 100,000. And
> then we should also assign a cost to the act of relaying a tuple from
> the parallel worker to the master, maybe cpu_tuple_cost (or some new
> GUC). For a small relation, or a query with a LIMIT clause, the
> parallel startup cost will make starting a lot of workers look
> unattractive, but for bigger relations it will make sense from a cost
> perspective, which is exactly what we want.
>
> There are probably other important considerations based on goals for
> overall resource utilization, and also because at a certain point
> adding more workers won't help because the disk will be saturated. I
> don't know exactly what we should do about those issues yet, but the
> steps described in the previous paragraph seem like a good place to
> start anyway.
>
Agreed.
>
> On Sat, Dec 6, 2014 at 7:07 AM, Stephen Frost <sfrost@snowman.net> wrote:
> > For my 2c, I'd like to see it support exactly what the SeqScan node
> > supports and then also what Foreign Scan supports. That would mean we'd
> > then be able to push filtering down to the workers which would be great.
> > Even better would be figuring out how to parallelize an Append node
> > (perhaps only possible when the nodes underneath are all SeqScan or
> > ForeignScan nodes) since that would allow us to then parallelize the
> > work across multiple tables and remote servers.
>
> I don't see how we can support the stuff ForeignScan does; presumably
> any parallelism there is up to the FDW to implement, using whatever
> in-core tools we provide. I do agree that parallelizing Append nodes
> is useful; but let's get one thing done first before we start trying
> to do thing #2.
>
> > I'm not entirely following this. How can the worker be responsible for
> > its own "plan" when the information passed to it (per the above
> > paragraph..) is pretty minimal? In general, I don't think we need to
> > have specifics like "this worker is going to do exactly X" because we
> > will eventually need some communication to happen between the worker and
> > the master process where the worker can ask for more work because it's
> > finished what it was tasked with and the master will need to give it
> > another chunk of work to do. I don't think we want exactly what each
> > worker process will do to be fully formed at the outset because, even
> > with the best information available, given concurrent load on the
> > system, it's not going to be perfect and we'll end up starving workers.
> > The plan, as formed by the master, should be more along the lines of
> > "this is what I'm gonna have my workers do" along w/ how many workers,
> > etc, and then it goes and does it. Perhaps for an 'explain analyze' we
> > return information about what workers actually *did* what, but that's a
> > whole different discussion.
>
> I agree with this. For a first version, I think it's OK to start a
> worker up for a particular sequential scan and have it help with that
> sequential scan until the scan is completed, and then exit. It should
> not, as the present version of the patch does, assign a fixed block
> range to each worker; instead, workers should allocate a block or
> chunk of blocks to work on until no blocks remain. That way, even if
> every worker but one gets stuck, the rest of the scan can still
> finish.
>
On Tue, Dec 9, 2014 at 12:46 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I agree with this. For a first version, I think it's OK to start a >> worker up for a particular sequential scan and have it help with that >> sequential scan until the scan is completed, and then exit. It should >> not, as the present version of the patch does, assign a fixed block >> range to each worker; instead, workers should allocate a block or >> chunk of blocks to work on until no blocks remain. That way, even if >> every worker but one gets stuck, the rest of the scan can still >> finish. >> > I will check on this point and see if it is feasible to do something on > those lines, basically currently at Executor initialization phase, we > set the scan limits and then during Executor Run phase use > heap_getnext to fetch the tuples accordingly, but doing it dynamically > means at ExecutorRun phase we need to reset the scan limit for > which page/pages to scan, still I have to check if there is any problem > with such an idea. Do you any different idea in mind? Hmm. Well, it looks like there are basically two choices: you can either (as you propose) deal with this above the level of the heap_beginscan/heap_getnext API by scanning one or a few pages at a time and then resetting the scan to a new starting page via heap_setscanlimits; or alternatively, you can add a callback to HeapScanDescData that, if non-NULL, will be invoked to get the next block number to scan. I'm not entirely sure which is better. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Sat, Dec 6, 2014 at 5:37 PM, Stephen Frost <sfrost@snowman.net> wrote:
> >
>
> So to summarize my understanding, below are the set of things
> which I should work on and in the order they are listed.
>
> 1. Push down qualification
> 2. Performance Data
> 3. Improve the way to push down the information related to worker.
> 4. Dynamic allocation of work for workers.
>
>
I have worked on the patch to accomplish above mentioned points
RAM = 492GB
Selection_criteria – 1% of rows will be selected
num_workers | exec_time (ms) |
0 | 229534 |
2 | 121741 |
4 | 67051 |
8 | 35607 |
16 | 24743 |
Selection_criteria – 10% of rows will be selected
num_workers | exec_time (ms) |
0 | 226671 |
2 | 151587 |
4 | 93648 |
8 | 70540 |
16 | 55466 |
num_workers | exec_time (ms) |
0 | 232673 |
2 | 197609 |
4 | 142686 |
8 | 111664 |
16 | 98097 |
Selection_criteria – 1% of rows will be selected
num_workers | exec_time (ms) |
0 | 15505 |
2 | 9155 |
4 | 6030 |
8 | 4523 |
16 | 4459 |
32 | 8259 |
64 | 13388 |
Selection_criteria – 5% of rows will be selected
num_workers | exec_time (ms) |
0 | 18906 |
2 | 13446 |
4 | 8970 |
8 | 7887 |
16 | 10403 |
Selection_criteria – 10% of rows will be selected
num_workers | exec_time (ms) |
0 | 16132 |
2 | 23780 |
4 | 20275 |
8 | 11390 |
16 | 11418 |
On Thu, Dec 18, 2014 at 9:22 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> On Mon, Dec 8, 2014 at 10:40 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > On Sat, Dec 6, 2014 at 5:37 PM, Stephen Frost <sfrost@snowman.net> wrote:
> > >
> >
> > So to summarize my understanding, below are the set of things
> > which I should work on and in the order they are listed.
> >
> > 1. Push down qualification
> > 2. Performance Data
> > 3. Improve the way to push down the information related to worker.
> > 4. Dynamic allocation of work for workers.
> >
> >
>
> I have worked on the patch to accomplish above mentioned points
> 1, 2 and partly 3 and would like to share the progress with community.
Attachment
Amit, * Amit Kapila (amit.kapila16@gmail.com) wrote: > 1. Parallel workers help a lot when there is an expensive qualification > to evaluated, the more expensive the qualification the more better are > results. I'd certainly hope so. ;) > 2. It works well for low selectivity quals and as the selectivity increases, > the benefit tends to go down due to additional tuple communication cost > between workers and master backend. I'm a bit sad to hear that the communication between workers and the master backend is already being a bottleneck. Now, that said, the box you're playing with looks to be pretty beefy and therefore the i/o subsystem might be particularly good, but generally speaking, it's a lot faster to move data in memory than it is to pull it off disk, and so I wouldn't expect the tuple communication between processes to really be the bottleneck... > 3. After certain point, increasing having more number of workers won't > help and rather have negative impact, refer Test-4. Yes, I see that too and it's also interesting- have you been able to identify why? What is the overhead (specifically) which is causing that? > I think as discussed previously we need to introduce 2 additional cost > variables (parallel_startup_cost, cpu_tuple_communication_cost) to > estimate the parallel seq scan cost so that when the tables are small > or selectivity is high, it should increase the cost of parallel plan. I agree that we need to figure out a way to cost out parallel plans, but I have doubts about these being the right way to do that. There has been quite a bit of literature regarding parallel execution and planning- have you had a chance to review anything along those lines? We certainly like to draw on previous experiences and analysis rather than trying to pave our own way. With these additional costs comes the consideration that we're looking for a wall-clock runtime proxy and therefore, while we need to add costs for parallel startup and tuple communication, we have to reduce the overall cost because of the parallelism or we'd never end up choosing a parallel plan. Is the thought to simply add up all the costs and then divide? Or perhaps to divide the cost of the actual plan but then add in the parallel startup cost and the tuple communication cost? Perhaps there has been prior discussion on these points but I'm thinking we need a README or similar which discusses all of this and includes any references out to academic papers or similar as appropriate. Thanks! Stephen
On Fri, Dec 19, 2014 at 7:51 AM, Stephen Frost <sfrost@snowman.net> wrote: >> 3. After certain point, increasing having more number of workers won't >> help and rather have negative impact, refer Test-4. > > Yes, I see that too and it's also interesting- have you been able to > identify why? What is the overhead (specifically) which is causing > that? Let's rewind. Amit's results show that, with a naive algorithm (pre-distributing equal-sized chunks of the relation to every worker) and a fairly-naive first cut at how to pass tuples around (I believe largely from what I did in pg_background) he can sequential-scan a table with 8 workers at 6.4 times the speed of a single process, and you're complaining because it's not efficient enough? It's a first draft! Be happy we got 6.4x, for crying out loud! The barrier to getting parallel sequential scan (or any parallel feature at all) committed is not going to be whether an 8-way scan is 6.4 times faster or 7.1 times faster or 7.8 times faster. It's going to be whether it's robust and won't break things. We should be focusing most of our effort here on identifying and fixing robustness problems. I'd vote to commit a feature like this with a 3x performance speedup if I thought it was robust enough. I'm not saying we shouldn't try to improve the performance here - we definitely should. But I don't think we should say, oh, an 8-way scan isn't good enough, we need a 16-way or 32-way scan in order for this to be efficient. That is getting your priorities quite mixed up. >> I think as discussed previously we need to introduce 2 additional cost >> variables (parallel_startup_cost, cpu_tuple_communication_cost) to >> estimate the parallel seq scan cost so that when the tables are small >> or selectivity is high, it should increase the cost of parallel plan. > > I agree that we need to figure out a way to cost out parallel plans, but > I have doubts about these being the right way to do that. There has > been quite a bit of literature regarding parallel execution and > planning- have you had a chance to review anything along those lines? > We certainly like to draw on previous experiences and analysis rather > than trying to pave our own way. I agree that it would be good to review the literature, but am not aware of anything relevant. Could you (or can anyone) provide some links? > With these additional costs comes the consideration that we're looking > for a wall-clock runtime proxy and therefore, while we need to add costs > for parallel startup and tuple communication, we have to reduce the > overall cost because of the parallelism or we'd never end up choosing a > parallel plan. Is the thought to simply add up all the costs and then > divide? Or perhaps to divide the cost of the actual plan but then add > in the parallel startup cost and the tuple communication cost? This has been discussed, on this thread. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Robert, * Robert Haas (robertmhaas@gmail.com) wrote: > On Fri, Dec 19, 2014 at 7:51 AM, Stephen Frost <sfrost@snowman.net> wrote: > >> 3. After certain point, increasing having more number of workers won't > >> help and rather have negative impact, refer Test-4. > > > > Yes, I see that too and it's also interesting- have you been able to > > identify why? What is the overhead (specifically) which is causing > > that? > > Let's rewind. Amit's results show that, with a naive algorithm > (pre-distributing equal-sized chunks of the relation to every worker) > and a fairly-naive first cut at how to pass tuples around (I believe > largely from what I did in pg_background) he can sequential-scan a > table with 8 workers at 6.4 times the speed of a single process, and > you're complaining because it's not efficient enough? It's a first > draft! Be happy we got 6.4x, for crying out loud! He also showed cases where parallelizing a query even with just two workers caused a serious increase in the total runtime (Test 6). Even having four workers was slower in that case, but a modest performance improvment was reached at eight but then no improvement from that was seen when running with 16. Being able to understand what's happening will inform how we cost this to, hopefully, achieve the 6.4x gains where we can and avoid the pitfalls of performing worse than a single thread in cases where parallelism doesn't help. What would likely be very helpful in the analysis would be CPU time information- when running with eight workers, were we using 800% CPU (8x 100%), or something less (perhaps due to locking, i/o, or other processes). Perhaps it's my fault for not being surprised that a naive first cut gives us such gains as my experience with parallel operations and PG has generally been very good (through the use of multiple connections to the DB and therefore independent transactions, of course). I'm very excited that we're making such great progress towards having parallel execution in the DB as I've often used PG in data warehouse use-cases. > The barrier to getting parallel sequential scan (or any parallel > feature at all) committed is not going to be whether an 8-way scan is > 6.4 times faster or 7.1 times faster or 7.8 times faster. It's going > to be whether it's robust and won't break things. We should be > focusing most of our effort here on identifying and fixing robustness > problems. I'd vote to commit a feature like this with a 3x > performance speedup if I thought it was robust enough. I don't have any problem if an 8-way scan is 6.4x faster or if it's 7.1 times faster, but what if that 3x performance speedup is only achieved when running with 8 CPUs at 100%? We'd have to coach our users to constantly be tweaking the enable_parallel_query (or whatever) option for the queries where it helps and turning it off for others. I'm not so excited about that. > I'm not saying we shouldn't try to improve the performance here - we > definitely should. But I don't think we should say, oh, an 8-way scan > isn't good enough, we need a 16-way or 32-way scan in order for this > to be efficient. That is getting your priorities quite mixed up. I don't think I said that. What I was getting at is that we need a cost system which accounts for the costs accurately enough that we don't end up with worse performance than single-threaded operation. In general, I don't expect that to be very difficult and we can be conservative in the initial releases to hopefully avoid regressions, but it absolutely needs consideration. > >> I think as discussed previously we need to introduce 2 additional cost > >> variables (parallel_startup_cost, cpu_tuple_communication_cost) to > >> estimate the parallel seq scan cost so that when the tables are small > >> or selectivity is high, it should increase the cost of parallel plan. > > > > I agree that we need to figure out a way to cost out parallel plans, but > > I have doubts about these being the right way to do that. There has > > been quite a bit of literature regarding parallel execution and > > planning- have you had a chance to review anything along those lines? > > We certainly like to draw on previous experiences and analysis rather > > than trying to pave our own way. > > I agree that it would be good to review the literature, but am not > aware of anything relevant. Could you (or can anyone) provide some > links? There's certainly documentation available from the other RDBMS' which already support parallel query, as one source. Other academic papers exist (and once you've linked into one, the references and prior work helps bring in others). Sadly, I don't currently have ACM access (might have to change that..), but there are publicly available papers also, such as: http://i.stanford.edu/pub/cstr/reports/cs/tr/96/1570/CS-TR-96-1570.pdf http://www.vldb.org/conf/1998/p251.pdf http://www.cs.uiuc.edu/class/fa05/cs591han/sigmodpods04/sigmod/pdf/I-001c.pdf > > With these additional costs comes the consideration that we're looking > > for a wall-clock runtime proxy and therefore, while we need to add costs > > for parallel startup and tuple communication, we have to reduce the > > overall cost because of the parallelism or we'd never end up choosing a > > parallel plan. Is the thought to simply add up all the costs and then > > divide? Or perhaps to divide the cost of the actual plan but then add > > in the parallel startup cost and the tuple communication cost? > > This has been discussed, on this thread. Fantastic. What I found in the patch was: + /* + * We simply assume that cost will be equally shared by parallel + * workers which might not be true especially for doing disk access. + * XXX - We would like to change these values based on some concrete + * tests. + */ What I asked for was: ---- I'm thinking we need a README or similar which discusses all of this and includes any references out to academic papers or similar as appropriate. ---- Perhaps it doesn't deserve its own README, but we clearly need more. Thanks! Stephen
On 12/19/14 3:27 PM, Stephen Frost wrote: > We'd have to coach our users to > constantly be tweaking the enable_parallel_query (or whatever) option > for the queries where it helps and turning it off for others. I'm not > so excited about that. I'd be perfectly (that means 100%) happy if it just defaulted to off, but I could turn it up to 11 whenever I needed it. I don't believe to be the only one with this opinion, either. .marko
* Marko Tiikkaja (marko@joh.to) wrote: > On 12/19/14 3:27 PM, Stephen Frost wrote: > >We'd have to coach our users to > >constantly be tweaking the enable_parallel_query (or whatever) option > >for the queries where it helps and turning it off for others. I'm not > >so excited about that. > > I'd be perfectly (that means 100%) happy if it just defaulted to > off, but I could turn it up to 11 whenever I needed it. I don't > believe to be the only one with this opinion, either. Perhaps we should reconsider our general position on hints then and add them so users can define the plan to be used.. For my part, I don't see this as all that much different. Consider if we were just adding HashJoin support today as an example. Would we be happy if we had to default to enable_hashjoin = off? Or if users had to do that regularly because our costing was horrid? It's bad enough that we have to resort to those tweaks today in rare cases. Thanks, Stephen
On Fri, Dec 19, 2014 at 9:39 AM, Stephen Frost <sfrost@snowman.net> wrote: > Perhaps we should reconsider our general position on hints then and > add them so users can define the plan to be used.. For my part, I don't > see this as all that much different. > > Consider if we were just adding HashJoin support today as an example. > Would we be happy if we had to default to enable_hashjoin = off? Or if > users had to do that regularly because our costing was horrid? It's bad > enough that we have to resort to those tweaks today in rare cases. If you're proposing that it is not reasonable to have a GUC that limits the degree of parallelism, then I think that's outright crazy: that is probably the very first GUC we need to add. New query processing capabilities can entail new controlling GUCs, and parallelism, being as complex at it is, will probably add several of them. But the big picture here is that if you want to ever have parallelism in PostgreSQL at all, you're going to have to live with the first version being pretty crude. I think it's quite likely that the first version of parallel sequential scan will be just as buggy as Hot Standby was when we first added it, or as buggy as the multi-xact code was when it went in, and probably subject to an even greater variety of taxing limitations than any feature we've committed in the 6 years I've been involved in the project. We get to pick between that and not having it at all. I'll take a look at the papers you sent about parallel query optimization, but personally I think that's putting the cart not only before the horse but also before the road. For V1, we need a query optimization model that does not completely suck - no more. The key criterion here is that this has to WORK. There will be time enough to improve everything else once we reach that goal. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 12/19/2014 04:39 PM, Stephen Frost wrote: > * Marko Tiikkaja (marko@joh.to) wrote: >> On 12/19/14 3:27 PM, Stephen Frost wrote: >>> We'd have to coach our users to >>> constantly be tweaking the enable_parallel_query (or whatever) option >>> for the queries where it helps and turning it off for others. I'm not >>> so excited about that. >> >> I'd be perfectly (that means 100%) happy if it just defaulted to >> off, but I could turn it up to 11 whenever I needed it. I don't >> believe to be the only one with this opinion, either. > > Perhaps we should reconsider our general position on hints then and > add them so users can define the plan to be used.. For my part, I don't > see this as all that much different. > > Consider if we were just adding HashJoin support today as an example. > Would we be happy if we had to default to enable_hashjoin = off? Or if > users had to do that regularly because our costing was horrid? It's bad > enough that we have to resort to those tweaks today in rare cases. This is somewhat different. Imagine that we achieve perfect parallelization, so that when you set enable_parallel_query=8, every query runs exactly 8x faster on an 8-core system, by using all eight cores. Now, you might still want to turn parallelization off, or at least set it to a lower setting, on an OLTP system. You might not want a single query to hog all CPUs to run one query faster; you'd want to leave some for other queries. In particular, if you run a mix of short transactions, and some background-like tasks that run for minutes or hours, you do not want to starve the short transactions by giving all eight CPUs to the background task. Admittedly, this is a rather crude knob to tune for such things, but it's quite intuitive to a DBA: how many CPU cores is one query allowed to utilize? And we don't really have anything better. In real life, there's always some overhead to parallelization, so that even if you can make one query run faster by doing it, you might hurt overall throughput. To some extent, it's a latency vs. throughput tradeoff, and it's quite reasonable to have a GUC for that because people have different priorities. - Heikki
On 20/12/14 03:54, Heikki Linnakangas wrote: > On 12/19/2014 04:39 PM, Stephen Frost wrote: >> * Marko Tiikkaja (marko@joh.to) wrote: >>> On 12/19/14 3:27 PM, Stephen Frost wrote: >>>> We'd have to coach our users to >>>> constantly be tweaking the enable_parallel_query (or whatever) option >>>> for the queries where it helps and turning it off for others. I'm not >>>> so excited about that. >>> >>> I'd be perfectly (that means 100%) happy if it just defaulted to >>> off, but I could turn it up to 11 whenever I needed it. I don't >>> believe to be the only one with this opinion, either. >> >> Perhaps we should reconsider our general position on hints then and >> add them so users can define the plan to be used.. For my part, I don't >> see this as all that much different. >> >> Consider if we were just adding HashJoin support today as an example. >> Would we be happy if we had to default to enable_hashjoin = off? Or if >> users had to do that regularly because our costing was horrid? It's bad >> enough that we have to resort to those tweaks today in rare cases. > > This is somewhat different. Imagine that we achieve perfect > parallelization, so that when you set enable_parallel_query=8, every > query runs exactly 8x faster on an 8-core system, by using all eight > cores. > > Now, you might still want to turn parallelization off, or at least set > it to a lower setting, on an OLTP system. You might not want a single > query to hog all CPUs to run one query faster; you'd want to leave > some for other queries. In particular, if you run a mix of short > transactions, and some background-like tasks that run for minutes or > hours, you do not want to starve the short transactions by giving all > eight CPUs to the background task. > > Admittedly, this is a rather crude knob to tune for such things, > but it's quite intuitive to a DBA: how many CPU cores is one query > allowed to utilize? And we don't really have anything better. > > In real life, there's always some overhead to parallelization, so that > even if you can make one query run faster by doing it, you might hurt > overall throughput. To some extent, it's a latency vs. throughput > tradeoff, and it's quite reasonable to have a GUC for that because > people have different priorities. > > - Heikki > > > How about 3 numbers: minCPUs # > 0 maxCPUs # >= minCPUs fractionOfCPUs # rounded up If you just have the /*number*/ of CPUs then a setting that is appropriate for quad core, may be too /*small*/ for an octo core processor. If you just have the /*fraction*/ of CPUs then a setting that is appropriate for quad core, may be too /*large*/ for an octo core processor. Cheers, Gavin
Robert, * Robert Haas (robertmhaas@gmail.com) wrote: > On Fri, Dec 19, 2014 at 9:39 AM, Stephen Frost <sfrost@snowman.net> wrote: > > Perhaps we should reconsider our general position on hints then and > > add them so users can define the plan to be used.. For my part, I don't > > see this as all that much different. > > > > Consider if we were just adding HashJoin support today as an example. > > Would we be happy if we had to default to enable_hashjoin = off? Or if > > users had to do that regularly because our costing was horrid? It's bad > > enough that we have to resort to those tweaks today in rare cases. > > If you're proposing that it is not reasonable to have a GUC that > limits the degree of parallelism, then I think that's outright crazy: I'm pretty sure that I didn't say anything along those lines. I'll try to be clearer. What I'd like is such a GUC that we can set at a reasonable default of, say, 4, and trust that our planner will generally do the right thing. Clearly, this may be something which admins have to tweak but what I would really like to avoid is users having to set this GUC explicitly for each of their queries. > that is probably the very first GUC we need to add. New query > processing capabilities can entail new controlling GUCs, and > parallelism, being as complex at it is, will probably add several of > them. That's fine if they're intended for debugging issues or dealing with unexpected bugs or issues, but let's not go into this thinking we should add GUCs which are geared with the expectation of users tweaking them regularly. > But the big picture here is that if you want to ever have parallelism > in PostgreSQL at all, you're going to have to live with the first > version being pretty crude. I think it's quite likely that the first > version of parallel sequential scan will be just as buggy as Hot > Standby was when we first added it, or as buggy as the multi-xact code > was when it went in, and probably subject to an even greater variety > of taxing limitations than any feature we've committed in the 6 years > I've been involved in the project. We get to pick between that and > not having it at all. If it's disabled by default then I'm worried it won't really improve until it is. Perhaps that's setting a higher bar than you feel is necessary but, for my part at least, it doesn't feel like a very high level. > I'll take a look at the papers you sent about parallel query > optimization, but personally I think that's putting the cart not only > before the horse but also before the road. For V1, we need a query > optimization model that does not completely suck - no more. The key > criterion here is that this has to WORK. There will be time enough to > improve everything else once we reach that goal. I agree that it's got to work, but it also needs to be generally well designed, and have the expectation of being on by default. Thanks, Stephen
* Heikki Linnakangas (hlinnakangas@vmware.com) wrote: > On 12/19/2014 04:39 PM, Stephen Frost wrote: > >* Marko Tiikkaja (marko@joh.to) wrote: > >>I'd be perfectly (that means 100%) happy if it just defaulted to > >>off, but I could turn it up to 11 whenever I needed it. I don't > >>believe to be the only one with this opinion, either. > > > >Perhaps we should reconsider our general position on hints then and > >add them so users can define the plan to be used.. For my part, I don't > >see this as all that much different. > > > >Consider if we were just adding HashJoin support today as an example. > >Would we be happy if we had to default to enable_hashjoin = off? Or if > >users had to do that regularly because our costing was horrid? It's bad > >enough that we have to resort to those tweaks today in rare cases. > > This is somewhat different. Imagine that we achieve perfect > parallelization, so that when you set enable_parallel_query=8, every > query runs exactly 8x faster on an 8-core system, by using all eight > cores. To be clear, as I mentioned to Robert just now, I'm not objecting to a GUC being added to turn off or control parallelization. I don't want such a GUC to be a crutch for us to lean on when it comes to questions about the optimizer though. We need to work through the optimizer questions of "should this be parallelized?" and, perhaps later, "how many ways is it sensible to parallelize this?" I'm worried we'll take such a GUC as a directive along the lines of "we are being told to parallelize to exactly this level every time and for every query which can be." The GUC should be an input into the planner/optimizer much the way enable_hashjoin is, unless it's being done as a *limiting* factor for the administrator to be able to control, but we've generally avoided doing that (see: work_mem) and, if we're going to start, we should probably come up with an approach that addresses the considerations for other resources too. Thanks, Stephen
>
> Amit,
>
> * Amit Kapila (amit.kapila16@gmail.com) wrote:
> > 1. Parallel workers help a lot when there is an expensive qualification
> > to evaluated, the more expensive the qualification the more better are
> > results.
>
> I'd certainly hope so. ;)
>
> > 2. It works well for low selectivity quals and as the selectivity increases,
> > the benefit tends to go down due to additional tuple communication cost
> > between workers and master backend.
>
> I'm a bit sad to hear that the communication between workers and the
> master backend is already being a bottleneck. Now, that said, the box
> you're playing with looks to be pretty beefy and therefore the i/o
> subsystem might be particularly good, but generally speaking, it's a lot
> faster to move data in memory than it is to pull it off disk, and so I
> wouldn't expect the tuple communication between processes to really be
> the bottleneck...
>
> > 3. After certain point, increasing having more number of workers won't
> > help and rather have negative impact, refer Test-4.
>
> Yes, I see that too and it's also interesting- have you been able to
> identify why? What is the overhead (specifically) which is causing
> that?
>
> > I think as discussed previously we need to introduce 2 additional cost
> > variables (parallel_startup_cost, cpu_tuple_communication_cost) to
> > estimate the parallel seq scan cost so that when the tables are small
> > or selectivity is high, it should increase the cost of parallel plan.
>
> I agree that we need to figure out a way to cost out parallel plans, but
> I have doubts about these being the right way to do that. There has
> been quite a bit of literature regarding parallel execution and
> planning- have you had a chance to review anything along those lines?
> than trying to pave our own way.
>
> With these additional costs comes the consideration that we're looking
> for a wall-clock runtime proxy and therefore, while we need to add costs
> for parallel startup and tuple communication, we have to reduce the
> overall cost because of the parallelism or we'd never end up choosing a
> parallel plan. Is the thought to simply add up all the costs and then
> divide? Or perhaps to divide the cost of the actual plan but then add
> in the parallel startup cost and the tuple communication cost?
>
> Perhaps there has been prior discussion on these points but I'm thinking
> we need a README or similar which discusses all of this and includes any
> references out to academic papers or similar as appropriate.
>
On 12/21/14, 12:42 AM, Amit Kapila wrote: > On Fri, Dec 19, 2014 at 6:21 PM, Stephen Frost <sfrost@snowman.net <mailto:sfrost@snowman.net>> wrote: > a. Instead of passing value array, just pass tuple id, but retain the > buffer pin till master backend reads the tuple based on tupleid. > This has side effect that we have to retain buffer pin for longer > period of time, but again that might not have any problem in > real world usage of parallel query. > > b. Instead of passing value array, pass directly the tuple which could > be directly propagated by master backend to upper layer or otherwise > in master backend change some code such that it could propagate the > tuple array received via shared memory queue directly to frontend. > Basically save the one extra cycle of form/deform tuple. > > Both these need some new message type and handling for same in > Executor code. > > Having said above, I think we can try to optimize this in multiple > ways, however we need additional mechanism and changes in Executor > code which is error prone and doesn't seem to be important at this > stage where we want the basic feature to work. Would b require some means of ensuring we didn't try and pass raw tuples to frontends? Other than that potential wrinkle,it seems like less work than a. ... > I think there are mainly two things which can lead to benefit > by employing parallel workers > a. Better use of available I/O bandwidth > b. Better use of available CPU's by doing expression evaluation > by multiple workers. ... > In the above tests, it seems to me that the maximum benefit due to > 'a' is realized upto 4~8 workers I'd think a good first estimate here would be to just use effective_io_concurrency. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
>
> On 12/21/14, 12:42 AM, Amit Kapila wrote:
>>
>> On Fri, Dec 19, 2014 at 6:21 PM, Stephen Frost <sfrost@snowman.net <mailto:sfrost@snowman.net>> wrote:
>> a. Instead of passing value array, just pass tuple id, but retain the
>> buffer pin till master backend reads the tuple based on tupleid.
>> This has side effect that we have to retain buffer pin for longer
>> period of time, but again that might not have any problem in
>> real world usage of parallel query.
>>
>> b. Instead of passing value array, pass directly the tuple which could
>> be directly propagated by master backend to upper layer or otherwise
>> in master backend change some code such that it could propagate the
>> tuple array received via shared memory queue directly to frontend.
>> Basically save the one extra cycle of form/deform tuple.
>>
>> Both these need some new message type and handling for same in
>> Executor code.
>>
>> Having said above, I think we can try to optimize this in multiple
>> ways, however we need additional mechanism and changes in Executor
>> code which is error prone and doesn't seem to be important at this
>> stage where we want the basic feature to work.
>
>
> Would b require some means of ensuring we didn't try and pass raw tuples to frontends?
slot_getallattrs())
>
>
>> I think there are mainly two things which can lead to benefit
>> by employing parallel workers
>> a. Better use of available I/O bandwidth
>> b. Better use of available CPU's by doing expression evaluation
>> by multiple workers.
>
>
> ...
>
>> In the above tests, it seems to me that the maximum benefit due to
>> 'a' is realized upto 4~8 workers
>
>
> I'd think a good first estimate here would be to just use effective_io_concurrency.
>
On Thu, Dec 18, 2014 at 9:22 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> On Mon, Dec 8, 2014 at 10:40 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > On Sat, Dec 6, 2014 at 5:37 PM, Stephen Frost <sfrost@snowman.net> wrote:
> > >
> >
> > So to summarize my understanding, below are the set of things
> > which I should work on and in the order they are listed.
> >
> > 1. Push down qualification
> > 2. Performance Data
> > 3. Improve the way to push down the information related to worker.
> > 4. Dynamic allocation of work for workers.
> >
> >
>
> I have worked on the patch to accomplish above mentioned points
> 1, 2 and partly 3 and would like to share the progress with community.Sorry forgot to attach updated patch in last mail, attaching it now.
➤ psql://thom@[local]:5488/pgbench
# create table t1(c1 int, c2 char(500)) with (fillfactor=10);
CREATE TABLE
Time: 13.653 ms
➤ psql://thom@[local]:5488/pgbench
# insert into t1 values(generate_series(1,100),'amit');
INSERT 0 100
Time: 4.796 ms
➤ psql://thom@[local]:5488/pgbench
# explain select c1 from t1;
ERROR: could not register background process
HINT: You may need to increase max_worker_processes.
Time: 1.659 ms
➤ psql://thom@[local]:5488/pgbench
# show max_worker_processes ;
max_worker_processes
----------------------
8
(1 row)
Time: 0.199 ms
# show parallel_seqscan_degree ;
parallel_seqscan_degree
-------------------------
10
(1 row)
Should I really need to increase max_worker_processes to >= parallel_seqscan_degree? If so, shouldn't there be a hint here along with the error message pointing this out? And should the error be produced when only a *plan* is being requested?
# explain select distinct bid from pgbench_accounts;
QUERY PLAN
----------------------------------------------------------------------------------------
HashAggregate (cost=1446639.00..1446643.99 rows=499 width=4)
Group Key: pgbench_accounts.bid
-> Append (cost=0.00..1321639.00 rows=50000001 width=4)
-> Seq Scan on pgbench_accounts (cost=0.00..0.00 rows=1 width=4)
-> Seq Scan on pgbench_accounts_1 (cost=0.00..4279.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_2 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_3 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_4 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_5 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_6 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_7 (cost=0.00..2640.00 rows=100000 width=4)
...
-> Seq Scan on pgbench_accounts_498 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_499 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_500 (cost=0.00..2640.00 rows=100000 width=4)
(504 rows)
On 18 December 2014 at 16:03, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Dec 18, 2014 at 9:22 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> On Mon, Dec 8, 2014 at 10:40 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > On Sat, Dec 6, 2014 at 5:37 PM, Stephen Frost <sfrost@snowman.net> wrote:
> > >
> >
> > So to summarize my understanding, below are the set of things
> > which I should work on and in the order they are listed.
> >
> > 1. Push down qualification
> > 2. Performance Data
> > 3. Improve the way to push down the information related to worker.
> > 4. Dynamic allocation of work for workers.
> >
> >
>
> I have worked on the patch to accomplish above mentioned points
> 1, 2 and partly 3 and would like to share the progress with community.Sorry forgot to attach updated patch in last mail, attaching it now.When attempting to recreate the plan in your example, I get an error:
➤ psql://thom@[local]:5488/pgbench
# create table t1(c1 int, c2 char(500)) with (fillfactor=10);
CREATE TABLE
Time: 13.653 ms
➤ psql://thom@[local]:5488/pgbench
# insert into t1 values(generate_series(1,100),'amit');
INSERT 0 100
Time: 4.796 ms
➤ psql://thom@[local]:5488/pgbench
# explain select c1 from t1;
ERROR: could not register background process
HINT: You may need to increase max_worker_processes.
Time: 1.659 ms
➤ psql://thom@[local]:5488/pgbench
# show max_worker_processes ;
max_worker_processes
----------------------
8
(1 row)
Time: 0.199 ms
# show parallel_seqscan_degree ;
parallel_seqscan_degree
-------------------------
10
(1 row)
Should I really need to increase max_worker_processes to >= parallel_seqscan_degree? If so, shouldn't there be a hint here along with the error message pointing this out? And should the error be produced when only a *plan* is being requested?Also, I noticed that where a table is partitioned, the plan isn't parallelised:
# explain select distinct bid from pgbench_accounts;
QUERY PLAN
----------------------------------------------------------------------------------------
HashAggregate (cost=1446639.00..1446643.99 rows=499 width=4)
Group Key: pgbench_accounts.bid
-> Append (cost=0.00..1321639.00 rows=50000001 width=4)
-> Seq Scan on pgbench_accounts (cost=0.00..0.00 rows=1 width=4)
-> Seq Scan on pgbench_accounts_1 (cost=0.00..4279.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_2 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_3 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_4 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_5 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_6 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_7 (cost=0.00..2640.00 rows=100000 width=4)
...
-> Seq Scan on pgbench_accounts_498 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_499 (cost=0.00..2640.00 rows=100000 width=4)
-> Seq Scan on pgbench_accounts_500 (cost=0.00..2640.00 rows=100000 width=4)
(504 rows)Is this expected?
➤ psql://thom@[local]:5488/pgbench2
# explain select distinct bid from pgbench_accounts;
QUERY PLAN
-------------------------------------------------------------------------------------------
HashAggregate (cost=245833.38..245834.38 rows=100 width=4)
Group Key: bid
-> Parallel Seq Scan on pgbench_accounts (cost=0.00..220833.38 rows=10000000 width=4)
Number of Workers: 8
Number of Blocks Per Workers: 208333
(5 rows)
Time: 7.476 ms
➤ psql://thom@[local]:5488/pgbench2
# explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
server closed the connection unexpectedly
This probably means the server terminated abnormally
before or while processing the request.
The connection to the server was lost. Attempting reset: Failed.
Time: 14897.991 ms
2014-12-31 15:21:42 GMT [9164]: [240-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [241-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [242-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [243-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [244-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [245-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [246-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [247-1] user=,db=,client= LOG: registering background worker "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [248-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [249-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [250-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [251-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [252-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [253-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [254-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:42 GMT [9164]: [255-1] user=,db=,client= LOG: starting background worker process "backend_worker"
2014-12-31 15:21:46 GMT [9164]: [256-1] user=,db=,client= LOG: worker process: backend_worker (PID 10887) exited with exit code 1
2014-12-31 15:21:46 GMT [9164]: [257-1] user=,db=,client= LOG: unregistering background worker "backend_worker"
2014-12-31 15:21:50 GMT [9164]: [258-1] user=,db=,client= LOG: worker process: backend_worker (PID 10888) exited with exit code 1
2014-12-31 15:21:50 GMT [9164]: [259-1] user=,db=,client= LOG: unregistering background worker "backend_worker"
2014-12-31 15:21:57 GMT [9164]: [260-1] user=,db=,client= LOG: server process (PID 10869) was terminated by signal 9: Killed
2014-12-31 15:21:57 GMT [9164]: [261-1] user=,db=,client= DETAIL: Failed process was running: explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
2014-12-31 15:21:57 GMT [9164]: [262-1] user=,db=,client= LOG: terminating any other active server processes
>
>
> When attempting to recreate the plan in your example, I get an error:
>
> ➤ psql://thom@[local]:5488/pgbench
>
> # create table t1(c1 int, c2 char(500)) with (fillfactor=10);
> CREATE TABLE
> Time: 13.653 ms
>
> ➤ psql://thom@[local]:5488/pgbench
>
> # insert into t1 values(generate_series(1,100),'amit');
> INSERT 0 100
> Time: 4.796 ms
>
> ➤ psql://thom@[local]:5488/pgbench
>
> # explain select c1 from t1;
> ERROR: could not register background process
> HINT: You may need to increase max_worker_processes.
> Time: 1.659 ms
>
> ➤ psql://thom@[local]:5488/pgbench
>
> # show max_worker_processes ;
> max_worker_processes
> ----------------------
> 8
> (1 row)
>
> Time: 0.199 ms
>
> # show parallel_seqscan_degree ;
> parallel_seqscan_degree
> -------------------------
> 10
> (1 row)
>
>
> Should I really need to increase max_worker_processes to >= parallel_seqscan_degree?
> Also, I noticed that where a table is partitioned, the plan isn't parallelised:
>
>
> Is this expected?
>
>
> Another issue (FYI, pgbench2 initialised with: pgbench -i -s 100 -F 10 pgbench2):
> ➤ psql://thom@[local]:5488/pgbench2
>
> # explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
> server closed the connection unexpectedly
> This probably means the server terminated abnormally
> before or while processing the request.
> The connection to the server was lost. Attempting reset: Failed.
> Time: 14897.991 ms
>
> 2014-12-31 15:21:57 GMT [9164]: [260-1] user=,db=,client= LOG: server process (PID 10869) was terminated by signal 9: Killed
> 2014-12-31 15:21:57 GMT [9164]: [261-1] user=,db=,client= DETAIL: Failed process was running: explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
> 2014-12-31 15:21:57 GMT [9164]: [262-1] user=,db=,client= LOG: terminating any other active server processes
>
> Running it again, I get the same issue. This is with parallel_seqscan_degree set to 8, and the crash occurs with 4 and 2 too.
>
> This doesn't happen if I set the pgbench scale to 50. I suspect this is a OOM issue. My laptop has 16GB RAM, the table is around 13GB at scale 100, and I don't have swap enabled. But I'm concerned it crashes the whole instance.
>
> I also notice that requesting BUFFERS in a parallel EXPLAIN output yields no such information.
> --
Yeah and the reason for same is that all the work done related
I think one thing we could do minimize the chance of such anerror is set the value of parallel workers to be used for plan equalto max_worker_processes if parallel_seqscan_degree is greaterthan max_worker_processes. Even if we do this, still such anerror can come if user has registered bgworker before we couldstart parallel plan execution.
--
Consultoria/Coaching PostgreSQL
>> Blog: http://fabriziomello.github.io
>> Linkedin: http://br.linkedin.com/in/fabriziomello
>> Twitter: http://twitter.com/fabriziomello
On Thu, Jan 1, 2015 at 12:00 PM, Fabrízio de Royes Mello <fabriziomello@gmail.com> wrote: > Can we check the number of free bgworkers slots to set the max workers? The real solution here is that this patch can't throw an error if it's unable to obtain the desired number of background workers. It needs to be able to smoothly degrade to a smaller number of background workers, or none at all. I think a lot of this work will fall out quite naturally if this patch is reworked to use the parallel mode/parallel context stuff, the latest version of which includes an example of how to set up a parallel scan in such a manner that it can run with any number of workers. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Jan 1, 2015 at 12:00 PM, Fabrízio de Royes Mello
<fabriziomello@gmail.com> wrote:
> Can we check the number of free bgworkers slots to set the max workers?
The real solution here is that this patch can't throw an error if it's
unable to obtain the desired number of background workers. It needs
to be able to smoothly degrade to a smaller number of background
workers, or none at all. I think a lot of this work will fall out
quite naturally if this patch is reworked to use the parallel
mode/parallel context stuff, the latest version of which includes an
example of how to set up a parallel scan in such a manner that it can
run with any number of workers.
That sounds like exactly what's needed.
>
> On Thu, Jan 1, 2015 at 12:00 PM, Fabrízio de Royes Mello
> <fabriziomello@gmail.com> wrote:
> > Can we check the number of free bgworkers slots to set the max workers?
>
> The real solution here is that this patch can't throw an error if it's
> unable to obtain the desired number of background workers. It needs
> to be able to smoothly degrade to a smaller number of background
> workers, or none at all.
> Running it again, I get the same issue. This is with parallel_seqscan_degree set to 8, and the crash occurs with 4 and 2 too.>
> This doesn't happen if I set the pgbench scale to 50. I suspect this is a OOM issue. My laptop has 16GB RAM, the table is around 13GB at scale 100, and I don't have swap enabled. But I'm concerned it crashes the whole instance.
>Isn't this a backend crash due to OOM?And after that server will restart automatically.
> I also notice that requesting BUFFERS in a parallel EXPLAIN output yields no such information.
> --
Yeah and the reason for same is that all the work done relatedto BUFFERS is done by backend workers, master backenddoesn't read any pages, so it is not able to accumulate thisinformation.> Is that not possible to report?It is not impossible to report such information, we can develop someway to share such information between master backend and workers.I think we can do this if required once the patch is more stablized.
And will the planner be able to decide whether or not it'll choose to use background workers or not? For example:
# explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------
HashAggregate (cost=89584.00..89584.05 rows=5 width=4) (actual time=228.222..228.224 rows=5 loops=1)
Output: bid
Group Key: pgbench_accounts.bid
Buffers: shared hit=83334
-> Seq Scan on public.pgbench_accounts (cost=0.00..88334.00 rows=500000 width=4) (actual time=0.008..136.522 rows=500000 loops=1)
Output: bid
Buffers: shared hit=83334
Planning time: 0.071 ms
Execution time: 228.265 ms
(9 rows)
# set parallel_seqscan_degree = 8;
SET
Time: 0.187 ms
# explain (analyse, buffers, verbose) select distinct bid from pgbench_accounts;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------------------------------
HashAggregate (cost=12291.75..12291.80 rows=5 width=4) (actual time=603.042..603.042 rows=1 loops=1)
Output: bid
Group Key: pgbench_accounts.bid
-> Parallel Seq Scan on public.pgbench_accounts (cost=0.00..11041.75 rows=500000 width=4) (actual time=2.445..529.284 rows=500000 loops=1)
Output: bid
Number of Workers: 8
Number of Blocks Per Workers: 10416
Planning time: 0.049 ms
Execution time: 663.103 ms
(9 rows)
Time: 663.437 ms
>
> On 1 January 2015 at 10:34, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> > Running it again, I get the same issue. This is with parallel_seqscan_degree set to 8, and the crash occurs with 4 and 2 too.
>> >
>> > This doesn't happen if I set the pgbench scale to 50. I suspect this is a OOM issue. My laptop has 16GB RAM, the table is around 13GB at scale 100, and I don't have swap enabled. But I'm concerned it crashes the whole instance.
>> >
>>
>> Isn't this a backend crash due to OOM?
>> And after that server will restart automatically.
>
>
> Yes, I'm fairly sure it is. I guess what I'm confused about is that 8 parallel sequential scans in separate sessions (1 per session) don't cause the server to crash, but in a single session (8 in 1 session), they do.
>
It could be possible that master backend retains some memory
>
> Will there be a GUC to influence parallel scan cost? Or does it take into account effective_io_concurrency in the costs?
>
>
Yes, we are planing to introduce cost model for parallel
On Fri, Jan 2, 2015 at 4:09 PM, Thom Brown <thom@linux.com> wrote:
>
> On 1 January 2015 at 10:34, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> > Running it again, I get the same issue. This is with parallel_seqscan_degree set to 8, and the crash occurs with 4 and 2 too.
>> >
>> > This doesn't happen if I set the pgbench scale to 50. I suspect this is a OOM issue. My laptop has 16GB RAM, the table is around 13GB at scale 100, and I don't have swap enabled. But I'm concerned it crashes the whole instance.
>> >
>>
>> Isn't this a backend crash due to OOM?
>> And after that server will restart automatically.
>
>
> Yes, I'm fairly sure it is. I guess what I'm confused about is that 8 parallel sequential scans in separate sessions (1 per session) don't cause the server to crash, but in a single session (8 in 1 session), they do.
>
It could be possible that master backend retains some memoryfor longer period which causes it to hit OOM error, by the wayin your test does always master backend hits OOM or is itrandom (either master or worker)
>
> Will there be a GUC to influence parallel scan cost? Or does it take into account effective_io_concurrency in the costs?
>> And will the planner be able to decide whether or not it'll choose to use background workers or not? For example:
>
Yes, we are planing to introduce cost model for parallelcommunication (there is some discussion about the sameupthread), but it's still not there and that's why youare seeing it to choose parallel plan when it shouldn't.Currently in patch, if you set parallel_seqscan_degree, itwill most probably choose parallel plan only.
Thanks.
On Fri, Jan 2, 2015 at 5:36 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Jan 1, 2015 at 11:29 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> On Thu, Jan 1, 2015 at 12:00 PM, Fabrízio de Royes Mello >> <fabriziomello@gmail.com> wrote: >> > Can we check the number of free bgworkers slots to set the max workers? >> >> The real solution here is that this patch can't throw an error if it's >> unable to obtain the desired number of background workers. It needs >> to be able to smoothly degrade to a smaller number of background >> workers, or none at all. > > I think handling this way can have one side effect which is that if > we degrade to smaller number, then the cost of plan (which was > decided by optimizer based on number of parallel workers) could > be more than non-parallel scan. > Ideally before finalizing the parallel plan we should reserve the > bgworkers required to execute that plan, but I think as of now > we can workout a solution without it. I don't think this is very practical. When cached plans are in use, we can have a bunch of plans sitting around that may or may not get reused at some point in the future, possibly far in the future. The current situation, which I think we want to maintain, is that such plans hold no execution-time resources (e.g. locks) and, generally, don't interfere with other things people might want to execute on the system. Nailing down a bunch of background workers just in case we might want to use them in the future would be pretty unfriendly. I think it's right to view this in the same way we view work_mem. We plan on the assumption that an amount of memory equal to work_mem will be available at execution time, without actually reserving it. If the plan happens to need that amount of memory and if it actually isn't available when needed, then performance will suck; conceivably, the OOM killer might trigger. But it's the user's job to avoid this by not setting work_mem too high in the first place. Whether this system is for the best is arguable: one can certainly imagine a system where, if there's not enough memory at execution time, we consider alternatives like (a) replanning with a lower memory target, (b) waiting until more memory is available, or (c) failing outright in lieu of driving the machine into swap. But devising such a system is complicated -- for example, replanning with a lower memory target might be latch onto a far more expensive plan, such that we would have been better off waiting for more memory to be available; yet trying to waiting until more memory is available might result in waiting forever. And that's why we don't have such a system. We don't need to do any better here. The GUC should tell us how many parallel workers we should anticipate being able to obtain. If other settings on the system, or the overall system load, preclude us from obtaining that number of parallel workers, then the query will take longer to execute; and the plan might be sub-optimal. If that happens frequently, the user should lower the planner GUC to a level that reflects the resources actually likely to be available at execution time. By the way, another area where this kind of effect crops up is with the presence of particular disk blocks in shared_buffers or the system buffer cache. Right now, the planner makes no attempt to cost a scan of a frequently-used, fully-cached relation different than a rarely-used, probably-not-cached relation; and that sometimes leads to bad plans. But if it did try to do that, then we'd have the same kind of problem discussed here -- things might change between planning and execution, or even after the beginning of execution. Also, we might get nasty feedback effects: since the relation isn't cached, we view a plan that would involve reading it in as very expensive, and avoid such a plan. However, we might be better off picking the "slow" plan anyway, because it might be that once we've read the data once it will stay cached and run much more quickly than some plan that seems better starting from a cold cache. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
* Robert Haas (robertmhaas@gmail.com) wrote: > I think it's right to view this in the same way we view work_mem. We > plan on the assumption that an amount of memory equal to work_mem will > be available at execution time, without actually reserving it. Agreed- this seems like a good approach for how to address this. We should still be able to end up with plans which use less than the max possible parallel workers though, as I pointed out somewhere up-thread. This is also similar to work_mem- we certainly have plans which don't expect to use all of work_mem and others that expect to use all of it (per node, of course). Thanks, Stephen
>
> On Fri, Jan 2, 2015 at 5:36 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Thu, Jan 1, 2015 at 11:29 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >> On Thu, Jan 1, 2015 at 12:00 PM, Fabrízio de Royes Mello
> >> <fabriziomello@gmail.com> wrote:
> >> > Can we check the number of free bgworkers slots to set the max workers?
> >>
> >> The real solution here is that this patch can't throw an error if it's
> >> unable to obtain the desired number of background workers. It needs
> >> to be able to smoothly degrade to a smaller number of background
> >> workers, or none at all.
> >
> > I think handling this way can have one side effect which is that if
> > we degrade to smaller number, then the cost of plan (which was
> > decided by optimizer based on number of parallel workers) could
> > be more than non-parallel scan.
> > Ideally before finalizing the parallel plan we should reserve the
> > bgworkers required to execute that plan, but I think as of now
> > we can workout a solution without it.
>
> I don't think this is very practical. When cached plans are in use,
> we can have a bunch of plans sitting around that may or may not get
> reused at some point in the future, possibly far in the future. The
> current situation, which I think we want to maintain, is that such
> plans hold no execution-time resources (e.g. locks) and, generally,
> don't interfere with other things people might want to execute on the
> system. Nailing down a bunch of background workers just in case we
> might want to use them in the future would be pretty unfriendly.
>
> I think it's right to view this in the same way we view work_mem. We
> plan on the assumption that an amount of memory equal to work_mem will
> be available at execution time, without actually reserving it.
> plan happens to need that amount of memory and if it actually isn't
> available when needed, then performance will suck; conceivably, the
> OOM killer might trigger. But it's the user's job to avoid this by
> not setting work_mem too high in the first place. Whether this system
> is for the best is arguable: one can certainly imagine a system where,
> if there's not enough memory at execution time, we consider
> alternatives like (a) replanning with a lower memory target, (b)
> waiting until more memory is available, or (c) failing outright in
> lieu of driving the machine into swap. But devising such a system is
> complicated -- for example, replanning with a lower memory target
> might be latch onto a far more expensive plan, such that we would have
> been better off waiting for more memory to be available; yet trying to
> waiting until more memory is available might result in waiting
> forever. And that's why we don't have such a system.
>
> We don't need to do any better here. The GUC should tell us how many
> parallel workers we should anticipate being able to obtain. If other
> settings on the system, or the overall system load, preclude us from
> obtaining that number of parallel workers, then the query will take
> longer to execute; and the plan might be sub-optimal. If that happens
> frequently, the user should lower the planner GUC to a level that
> reflects the resources actually likely to be available at execution
> time.
>
>
> On Mon, Jan 5, 2015 at 8:31 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
Sorry for incomplete mail sent prior to this, I just hit the send button
4. Sending ReadyForQuery() after completely sending the tuples,
Attachment
On 1/5/15, 9:21 AM, Stephen Frost wrote: > * Robert Haas (robertmhaas@gmail.com) wrote: >> I think it's right to view this in the same way we view work_mem. We >> plan on the assumption that an amount of memory equal to work_mem will >> be available at execution time, without actually reserving it. > > Agreed- this seems like a good approach for how to address this. We > should still be able to end up with plans which use less than the max > possible parallel workers though, as I pointed out somewhere up-thread. > This is also similar to work_mem- we certainly have plans which don't > expect to use all of work_mem and others that expect to use all of it > (per node, of course). I agree, but we should try and warn the user if they set parallel_seqscan_degree close to max_worker_processes, or at leastgive some indication of what's going on. This is something you could end up beating your head on wondering why it'snot working. Perhaps we could have EXPLAIN throw a warning if a plan is likely to get less than parallel_seqscan_degree number of workers. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
* Jim Nasby (Jim.Nasby@BlueTreble.com) wrote: > On 1/5/15, 9:21 AM, Stephen Frost wrote: > >* Robert Haas (robertmhaas@gmail.com) wrote: > >>I think it's right to view this in the same way we view work_mem. We > >>plan on the assumption that an amount of memory equal to work_mem will > >>be available at execution time, without actually reserving it. > > > >Agreed- this seems like a good approach for how to address this. We > >should still be able to end up with plans which use less than the max > >possible parallel workers though, as I pointed out somewhere up-thread. > >This is also similar to work_mem- we certainly have plans which don't > >expect to use all of work_mem and others that expect to use all of it > >(per node, of course). > > I agree, but we should try and warn the user if they set parallel_seqscan_degree close to max_worker_processes, or at leastgive some indication of what's going on. This is something you could end up beating your head on wondering why it'snot working. > > Perhaps we could have EXPLAIN throw a warning if a plan is likely to get less than parallel_seqscan_degree number of workers. Yeah, if we come up with a plan for X workers and end up not being able to spawn that many then I could see that being worth a warning or notice or something. Not sure what EXPLAIN has to do anything with it.. Thanks, Stephen
>
> On 1/5/15, 9:21 AM, Stephen Frost wrote:
>>
>> * Robert Haas (robertmhaas@gmail.com) wrote:
>>>
>>> I think it's right to view this in the same way we view work_mem. We
>>> plan on the assumption that an amount of memory equal to work_mem will
>>> be available at execution time, without actually reserving it.
>>
>>
>> Agreed- this seems like a good approach for how to address this. We
>> should still be able to end up with plans which use less than the max
>> possible parallel workers though, as I pointed out somewhere up-thread.
>> This is also similar to work_mem- we certainly have plans which don't
>> expect to use all of work_mem and others that expect to use all of it
>> (per node, of course).
>
>
> I agree, but we should try and warn the user if they set parallel_seqscan_degree close to max_worker_processes, or at least give some indication of what's going on. This is something you could end up beating your head on wondering why it's not working.
>
>
>
> There's certainly documentation available from the other RDBMS' which
> already support parallel query, as one source. Other academic papers
> exist (and once you've linked into one, the references and prior work
> helps bring in others). Sadly, I don't currently have ACM access (might
> have to change that..), but there are publicly available papers also,
* parallel_startup_cost - Cost of starting up parallel workers with default
Amit, * Amit Kapila (amit.kapila16@gmail.com) wrote: > On Fri, Dec 19, 2014 at 7:57 PM, Stephen Frost <sfrost@snowman.net> wrote: > > There's certainly documentation available from the other RDBMS' which > > already support parallel query, as one source. Other academic papers > > exist (and once you've linked into one, the references and prior work > > helps bring in others). Sadly, I don't currently have ACM access (might > > have to change that..), but there are publicly available papers also, > > I have gone through couple of papers and what some other databases > do in case of parallel sequential scan and here is brief summarization > of same and how I am planning to handle in the patch: Great, thanks! > Costing: > In one of the paper's [1] suggested by you, below is the summarisation: > a. Startup costs are negligible if processes can be reused > rather than created afresh. > b. Communication cost consists of the CPU cost of sending > and receiving messages. > c. Communication costs can exceed the cost of operators such > as scanning, joining or grouping > These findings lead to the important conclusion that > Query optimization should be concerned with communication costs > but not with startup costs. > > In our case as currently we don't have a mechanism to reuse parallel > workers, so we need to account for that cost as well. So based on that, > I am planing to add three new parameters cpu_tuple_comm_cost, > parallel_setup_cost, parallel_startup_cost > * cpu_tuple_comm_cost - Cost of CPU time to pass a tuple from worker > to master backend with default value > DEFAULT_CPU_TUPLE_COMM_COST as 0.1, this will be multiplied > with tuples expected to be selected > * parallel_setup_cost - Cost of setting up shared memory for parallelism > with default value as 100.0 > * parallel_startup_cost - Cost of starting up parallel workers with > default > value as 1000.0 multiplied by number of workers decided for scan. > > I will do some experiments to finalise the default values, but in general, > I feel developing cost model on above parameters is good. The parameters sound reasonable but I'm a bit worried about the way you're describing the implementation. Specifically this comment: "Cost of starting up parallel workers with default value as 1000.0 multiplied by number of workers decided for scan." That appears to imply that we'll decide on the number of workers, figure out the cost, and then consider "parallel" as one path and "not-parallel" as another. I'm worried that if I end up setting the max parallel workers to 32 for my big, beefy, mostly-single-user system then I'll actually end up not getting parallel execution because we'll always be including the full startup cost of 32 threads. For huge queries, it'll probably be fine, but there's a lot of room to parallelize things at levels less than 32 which we won't even consider. What I was advocating for up-thread was to consider multiple "parallel" paths and to pick whichever ends up being the lowest overall cost. The flip-side to that is increased planning time. Perhaps we can come up with an efficient way of working out where the break-point is based on the non-parallel cost and go at it from that direction instead of building out whole paths for each increment of parallelism. I'd really like to be able to set the 'max parallel' high and then have the optimizer figure out how many workers should actually be spawned for a given query. > Execution: > Most other databases does partition level scan for partition on > different disks by each individual parallel worker. However, > it seems amazon dynamodb [2] also works on something > similar to what I have used in patch which means on fixed > blocks. I think this kind of strategy seems better than dividing > the blocks at runtime because dividing randomly the blocks > among workers could lead to random scan for a parallel > sequential scan. Yeah, we also need to consider the i/o side of this, which will definitely be tricky. There are i/o systems out there which are faster than a single CPU and ones where a single CPU can manage multiple i/o channels. There are also cases where the i/o system handles sequential access nearly as fast as random and cases where sequential is much faster than random. Where we can get an idea of that distinction is with seq_page_cost vs. random_page_cost as folks running on SSDs tend to lower random_page_cost from the default to indicate that. > Also I find in whatever I have read (Oracle, dynamodb) that most > databases divide work among workers and master backend acts > as coordinator, atleast that's what I could understand. Yeah, I agree that's more typical. Robert's point that the master backend should participate is interesting but, as I recall, it was based on the idea that the master could finish faster than the worker- but if that's the case then we've planned it out wrong from the beginning. Thanks! Stephen
Amit, * Amit Kapila (amit.kapila16@gmail.com) wrote: > On Fri, Jan 9, 2015 at 1:02 AM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote: > > I agree, but we should try and warn the user if they set > > parallel_seqscan_degree close to max_worker_processes, or at least give > > some indication of what's going on. This is something you could end up > > beating your head on wondering why it's not working. > > Yet another way to handle the case when enough workers are not > available is to let user specify the desired minimum percentage of > requested parallel workers with parameter like > PARALLEL_QUERY_MIN_PERCENT. For example, if you specify > 50 for this parameter, then at least 50% of the parallel workers > requested for any parallel operation must be available in order for > the operation to succeed else it will give error. If the value is set to > null, then all parallel operations will proceed as long as at least two > parallel workers are available for processing. Ugh. I'm not a fan of this.. Based on how we're talking about modeling this, if we decide to parallelize at all, then we expect it to be a win. I don't like the idea of throwing an error if, at execution time, we end up not being able to actually get the number of workers we want- instead, we should degrade gracefully all the way back to serial, if necessary. Perhaps we should send a NOTICE or something along those lines to let the user know we weren't able to get the level of parallelization that the plan originally asked for, but I really don't like just throwing an error. Now, for debugging purposes, I could see such a parameter being available but it should default to 'off/never-fail'. Thanks, Stephen
On 01/09/2015 08:01 PM, Stephen Frost wrote: > Amit, > > * Amit Kapila (amit.kapila16@gmail.com) wrote: >> On Fri, Jan 9, 2015 at 1:02 AM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote: >>> I agree, but we should try and warn the user if they set >>> parallel_seqscan_degree close to max_worker_processes, or at least give >>> some indication of what's going on. This is something you could end up >>> beating your head on wondering why it's not working. >> >> Yet another way to handle the case when enough workers are not >> available is to let user specify the desired minimum percentage of >> requested parallel workers with parameter like >> PARALLEL_QUERY_MIN_PERCENT. For example, if you specify >> 50 for this parameter, then at least 50% of the parallel workers >> requested for any parallel operation must be available in order for >> the operation to succeed else it will give error. If the value is set to >> null, then all parallel operations will proceed as long as at least two >> parallel workers are available for processing. > > Ugh. I'm not a fan of this.. Based on how we're talking about modeling > this, if we decide to parallelize at all, then we expect it to be a win. > I don't like the idea of throwing an error if, at execution time, we end > up not being able to actually get the number of workers we want- > instead, we should degrade gracefully all the way back to serial, if > necessary. Perhaps we should send a NOTICE or something along those > lines to let the user know we weren't able to get the level of > parallelization that the plan originally asked for, but I really don't > like just throwing an error. yeah this seems like the the behaviour I would expect, if we cant get enough parallel workers we should just use as much as we can get. Everything else and especially erroring out will just cause random application failures and easy DoS vectors. I think all we need initially is being able to specify a "maximum number of workers per query" as well as a "maximum number of workers in total for parallel operations". > > Now, for debugging purposes, I could see such a parameter being > available but it should default to 'off/never-fail'. not sure what it really would be useful for - if I execute a query I would truely expect it to get answered - if it can be made faster if done in parallel thats nice but why would I want it to fail? Stefan
* Stefan Kaltenbrunner (stefan@kaltenbrunner.cc) wrote: > On 01/09/2015 08:01 PM, Stephen Frost wrote: > > Now, for debugging purposes, I could see such a parameter being > > available but it should default to 'off/never-fail'. > > not sure what it really would be useful for - if I execute a query I > would truely expect it to get answered - if it can be made faster if > done in parallel thats nice but why would I want it to fail? I was thinking for debugging only, though I'm not really sure why you'd need it if you get a NOTICE when you don't end up with all the workers you expect. Thanks, Stephen
On 1/9/15, 3:34 PM, Stephen Frost wrote: > * Stefan Kaltenbrunner (stefan@kaltenbrunner.cc) wrote: >> On 01/09/2015 08:01 PM, Stephen Frost wrote: >>> Now, for debugging purposes, I could see such a parameter being >>> available but it should default to 'off/never-fail'. >> >> not sure what it really would be useful for - if I execute a query I >> would truely expect it to get answered - if it can be made faster if >> done in parallel thats nice but why would I want it to fail? > > I was thinking for debugging only, though I'm not really sure why you'd > need it if you get a NOTICE when you don't end up with all the workers > you expect. Yeah, debugging is my concern as well. You're working on a query, you expect it to be using parallelism, and EXPLAIN is showingit's not. Now you're scratching your head. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
On 1/9/15, 11:24 AM, Stephen Frost wrote: > What I was advocating for up-thread was to consider multiple "parallel" > paths and to pick whichever ends up being the lowest overall cost. The > flip-side to that is increased planning time. Perhaps we can come up > with an efficient way of working out where the break-point is based on > the non-parallel cost and go at it from that direction instead of > building out whole paths for each increment of parallelism. I think at some point we'll need the ability to stop planning part-way through for queries producing really small estimates.If the first estimate you get is 1000 units, does it really make sense to do something like try every possiblejoin permutation, or attempt to parallelize? -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
> * Amit Kapila (amit.kapila16@gmail.com) wrote:
> > In our case as currently we don't have a mechanism to reuse parallel
> > workers, so we need to account for that cost as well. So based on that,
> > I am planing to add three new parameters cpu_tuple_comm_cost,
> > parallel_setup_cost, parallel_startup_cost
> > * cpu_tuple_comm_cost - Cost of CPU time to pass a tuple from worker
> > to master backend with default value
> > DEFAULT_CPU_TUPLE_COMM_COST as 0.1, this will be multiplied
> > with tuples expected to be selected
> > * parallel_setup_cost - Cost of setting up shared memory for parallelism
> > with default value as 100.0
> > * parallel_startup_cost - Cost of starting up parallel workers with
> > default
> > value as 1000.0 multiplied by number of workers decided for scan.
> >
> > I will do some experiments to finalise the default values, but in general,
> > I feel developing cost model on above parameters is good.
>
> The parameters sound reasonable but I'm a bit worried about the way
> you're describing the implementation. Specifically this comment:
>
> "Cost of starting up parallel workers with default value as 1000.0
> multiplied by number of workers decided for scan."
>
> That appears to imply that we'll decide on the number of workers, figure
> out the cost, and then consider "parallel" as one path and
> "not-parallel" as another. I'm worried that if I end up setting the max
> parallel workers to 32 for my big, beefy, mostly-single-user system then
> I'll actually end up not getting parallel execution because we'll always
> be including the full startup cost of 32 threads. For huge queries,
> it'll probably be fine, but there's a lot of room to parallelize things
> at levels less than 32 which we won't even consider.
>
> What I was advocating for up-thread was to consider multiple "parallel"
> paths and to pick whichever ends up being the lowest overall cost. The
> flip-side to that is increased planning time.
> with an efficient way of working out where the break-point is based on
> the non-parallel cost and go at it from that direction instead of
> building out whole paths for each increment of parallelism.
>
> I'd really like to be able to set the 'max parallel' high and then have
> the optimizer figure out how many workers should actually be spawned for
> a given query.
>
> > Execution:
> > Most other databases does partition level scan for partition on
> > different disks by each individual parallel worker. However,
> > it seems amazon dynamodb [2] also works on something
> > similar to what I have used in patch which means on fixed
> > blocks. I think this kind of strategy seems better than dividing
> > the blocks at runtime because dividing randomly the blocks
> > among workers could lead to random scan for a parallel
> > sequential scan.
>
> Yeah, we also need to consider the i/o side of this, which will
> definitely be tricky. There are i/o systems out there which are faster
> than a single CPU and ones where a single CPU can manage multiple i/o
> channels. There are also cases where the i/o system handles sequential
> access nearly as fast as random and cases where sequential is much
> faster than random. Where we can get an idea of that distinction is
> with seq_page_cost vs. random_page_cost as folks running on SSDs tend to
> lower random_page_cost from the default to indicate that.
>
I am not clear, do you expect anything different in execution strategy
>
> On 01/09/2015 08:01 PM, Stephen Frost wrote:
> > Amit,
> >
> > * Amit Kapila (amit.kapila16@gmail.com) wrote:
> >> On Fri, Jan 9, 2015 at 1:02 AM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
> >>> I agree, but we should try and warn the user if they set
> >>> parallel_seqscan_degree close to max_worker_processes, or at least give
> >>> some indication of what's going on. This is something you could end up
> >>> beating your head on wondering why it's not working.
> >>
> >> Yet another way to handle the case when enough workers are not
> >> available is to let user specify the desired minimum percentage of
> >> requested parallel workers with parameter like
> >> PARALLEL_QUERY_MIN_PERCENT. For example, if you specify
> >> 50 for this parameter, then at least 50% of the parallel workers
> >> requested for any parallel operation must be available in order for
> >> the operation to succeed else it will give error. If the value is set to
> >> null, then all parallel operations will proceed as long as at least two
> >> parallel workers are available for processing.
> >
> > Now, for debugging purposes, I could see such a parameter being
> > available but it should default to 'off/never-fail'.
>
> not sure what it really would be useful for - if I execute a query I
> would truely expect it to get answered - if it can be made faster if
> done in parallel thats nice but why would I want it to fail?
>
One usecase where I could imagine it to be useful is when the
* Amit Kapila (amit.kapila16@gmail.com) wrote: > At this moment if we can ensure that parallel plan should not be selected > for cases where it will perform poorly is more than enough considering > we have lots of other work left to even make any parallel operation work. The problem with this approach is that it doesn't consider any options between 'serial' and 'parallelize by factor X'. If the startup cost is 1000 and the factor is 32, then a seqscan which costs 31000 won't ever be parallelized, even though a factor of 8 would have parallelized it. You could forget about the per-process startup cost entirely, in fact, and simply say "only parallelize if it's more than X". Again, I don't like the idea of designing this with the assumption that the user dictates the right level of parallelization for each and every query. I'd love to go out and tell users "set the factor to the number of CPUs you have and we'll just use what makes sense." The same goes for max number of backends. If we set the parallel level to the number of CPUs and set the max backends to the same, then we end up with only one parallel query running at a time, ever. That's terrible. Now, we could set the parallel level lower or set the max backends higher, but either way we're going to end up either using less than we could or over-subscribing, neither of which is good. I agree that this makes it a bit different from work_mem, but in this case there's an overall max in the form of the maximum number of background workers. If we had something similar for work_mem, then we could set that higher and still trust the system to only use the amount of memory necessary (eg: a hashjoin doesn't use all available work_mem and neither does a sort, unless the set is larger than available memory). > > > Execution: > > > Most other databases does partition level scan for partition on > > > different disks by each individual parallel worker. However, > > > it seems amazon dynamodb [2] also works on something > > > similar to what I have used in patch which means on fixed > > > blocks. I think this kind of strategy seems better than dividing > > > the blocks at runtime because dividing randomly the blocks > > > among workers could lead to random scan for a parallel > > > sequential scan. > > > > Yeah, we also need to consider the i/o side of this, which will > > definitely be tricky. There are i/o systems out there which are faster > > than a single CPU and ones where a single CPU can manage multiple i/o > > channels. There are also cases where the i/o system handles sequential > > access nearly as fast as random and cases where sequential is much > > faster than random. Where we can get an idea of that distinction is > > with seq_page_cost vs. random_page_cost as folks running on SSDs tend to > > lower random_page_cost from the default to indicate that. > > > I am not clear, do you expect anything different in execution strategy > than what I have mentioned or does that sound reasonable to you? What I'd like is a way to figure out the right amount of CPU for each tablespace (0.25, 1, 2, 4, etc) and then use that many. Using a single CPU for each tablespace is likely to starve the CPU or starve the I/O system and I'm not sure if there's a way to address that. Note that I intentionally said tablespace there because that's how users can tell us what the different i/o channels are. I realize this ends up going beyond the current scope, but the parallel seqscan at the per relation level will only ever be using one i/o channel. It'd be neat if we could work out how fast that i/o channel is vs. the CPUs and determine how many CPUs are necessary to keep up with the i/o channel and then use more-or-less exactly that many for the scan. I agree that some of this can come later but I worry that starting out with a design that expects to always be told exactly how many CPUs to use when running a parallel query will be difficult to move away from later. Thanks, Stephen
On Thu, Jan 8, 2015 at 6:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Are we sure that in such cases we will consume work_mem during > execution? In cases of parallel_workers we are sure to an extent > that if we reserve the workers then we will use it during execution. > Nonetheless, I have proceded and integrated the parallel_seq scan > patch with v0.3 of parallel_mode patch posted by you at below link: > http://www.postgresql.org/message-id/CA+TgmoYmp_=XcJEhvJZt9P8drBgW-pDpjHxBhZA79+M4o-CZQA@mail.gmail.com That depends on the costing model. It makes no sense to do a parallel sequential scan on a small relation, because the user backend can scan the whole thing itself faster than the workers can start up. I suspect it may also be true that the useful amount of parallelism increases the larger the relation gets (but maybe not). > 2. To enable two types of shared memory queue's (error queue and > tuple queue), we need to ensure that we switch to appropriate queue > during communication of various messages from parallel worker > to master backend. There are two ways to do it > a. Save the information about error queue during startup of parallel > worker (ParallelMain()) and then during error, set the same (switch > to error queue in errstart() and switch back to tuple queue in > errfinish() and errstart() in case errstart() doesn't need to > propagate > error). > b. Do something similar as (a) for tuple queue in printtup or other > place > if any for non-error messages. > I think approach (a) is slightly better as compare to approach (b) as > we need to switch many times for tuple queue (for each tuple) and > there could be multiple places where we need to do the same. For now, > I have used approach (a) in Patch which needs some more work if we > agree on the same. I don't think you should be "switching" queues. The tuples should be sent to the tuple queue, and errors and notices to the error queue. > 3. As per current implementation of Parallel_seqscan, it needs to use > some information from parallel.c which was not exposed, so I have > exposed the same by moving it to parallel.h. Information that is required > is as follows: > ParallelWorkerNumber, FixedParallelState and shm keys - > This is used to decide the blocks that needs to be scanned. > We might change it in future the way parallel scan/work distribution > is done, but I don't see any harm in exposing this information. Hmm. I can see why ParallelWorkerNumber might need to be exposed, but the other stuff seems like it shouldn't be. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote: > Yeah, if we come up with a plan for X workers and end up not being able > to spawn that many then I could see that being worth a warning or notice > or something. Not sure what EXPLAIN has to do anything with it.. That seems mighty odd to me. If there are 8 background worker processes available, and you allow each session to use at most 4, then when there are >2 sessions trying to do parallelism at the same time, they might not all get their workers. Emitting a notice for that seems like it would be awfully chatty. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Jan 9, 2015 at 12:24 PM, Stephen Frost <sfrost@snowman.net> wrote: > The parameters sound reasonable but I'm a bit worried about the way > you're describing the implementation. Specifically this comment: > > "Cost of starting up parallel workers with default value as 1000.0 > multiplied by number of workers decided for scan." > > That appears to imply that we'll decide on the number of workers, figure > out the cost, and then consider "parallel" as one path and > "not-parallel" as another. [...] > I'd really like to be able to set the 'max parallel' high and then have > the optimizer figure out how many workers should actually be spawned for > a given query. +1. > Yeah, we also need to consider the i/o side of this, which will > definitely be tricky. There are i/o systems out there which are faster > than a single CPU and ones where a single CPU can manage multiple i/o > channels. There are also cases where the i/o system handles sequential > access nearly as fast as random and cases where sequential is much > faster than random. Where we can get an idea of that distinction is > with seq_page_cost vs. random_page_cost as folks running on SSDs tend to > lower random_page_cost from the default to indicate that. On my MacOS X system, I've already seen cases where my parallel_count module runs incredibly slowly some of the time. I believe that this is because having multiple workers reading the relation block-by-block at the same time causes the OS to fail to realize that it needs to do aggressive readahead. I suspect we're going to need to account for this somehow. > Yeah, I agree that's more typical. Robert's point that the master > backend should participate is interesting but, as I recall, it was based > on the idea that the master could finish faster than the worker- but if > that's the case then we've planned it out wrong from the beginning. So, if the workers have been started but aren't keeping up, the master should do nothing until they produce tuples rather than participating?That doesn't seem right. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Jan 8, 2015 at 6:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > 2. To enable two types of shared memory queue's (error queue and
> > tuple queue), we need to ensure that we switch to appropriate queue
> > during communication of various messages from parallel worker
> > to master backend. There are two ways to do it
> > a. Save the information about error queue during startup of parallel
> > worker (ParallelMain()) and then during error, set the same (switch
> > to error queue in errstart() and switch back to tuple queue in
> > errfinish() and errstart() in case errstart() doesn't need to
> > propagate
> > error).
> > b. Do something similar as (a) for tuple queue in printtup or other
> > place
> > if any for non-error messages.
> > I think approach (a) is slightly better as compare to approach (b) as
> > we need to switch many times for tuple queue (for each tuple) and
> > there could be multiple places where we need to do the same. For now,
> > I have used approach (a) in Patch which needs some more work if we
> > agree on the same.
>
> I don't think you should be "switching" queues. The tuples should be
> sent to the tuple queue, and errors and notices to the error queue.
>
> > 3. As per current implementation of Parallel_seqscan, it needs to use
> > some information from parallel.c which was not exposed, so I have
> > exposed the same by moving it to parallel.h. Information that is required
> > is as follows:
> > ParallelWorkerNumber, FixedParallelState and shm keys -
> > This is used to decide the blocks that needs to be scanned.
> > We might change it in future the way parallel scan/work distribution
> > is done, but I don't see any harm in exposing this information.
>
> Hmm. I can see why ParallelWorkerNumber might need to be exposed, but
> the other stuff seems like it shouldn't be.
>
It depends upon how we decide to achieve the scan of blocks
* Robert Haas (robertmhaas@gmail.com) wrote: > On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote: > > Yeah, if we come up with a plan for X workers and end up not being able > > to spawn that many then I could see that being worth a warning or notice > > or something. Not sure what EXPLAIN has to do anything with it.. > > That seems mighty odd to me. If there are 8 background worker > processes available, and you allow each session to use at most 4, then > when there are >2 sessions trying to do parallelism at the same time, > they might not all get their workers. Emitting a notice for that > seems like it would be awfully chatty. Yeah, agreed, it could get quite noisy. Did you have another thought for how to address the concern raised? Specifically, that you might not get as many workers as you thought you would? Thanks, Stephen
* Robert Haas (robertmhaas@gmail.com) wrote: > On Fri, Jan 9, 2015 at 12:24 PM, Stephen Frost <sfrost@snowman.net> wrote: > > Yeah, we also need to consider the i/o side of this, which will > > definitely be tricky. There are i/o systems out there which are faster > > than a single CPU and ones where a single CPU can manage multiple i/o > > channels. There are also cases where the i/o system handles sequential > > access nearly as fast as random and cases where sequential is much > > faster than random. Where we can get an idea of that distinction is > > with seq_page_cost vs. random_page_cost as folks running on SSDs tend to > > lower random_page_cost from the default to indicate that. > > On my MacOS X system, I've already seen cases where my parallel_count > module runs incredibly slowly some of the time. I believe that this > is because having multiple workers reading the relation block-by-block > at the same time causes the OS to fail to realize that it needs to do > aggressive readahead. I suspect we're going to need to account for > this somehow. So, for my 2c, I've long expected us to parallelize at the relation-file level for these kinds of operations. This goes back to my other thoughts on how we should be thinking about parallelizing inbound data for bulk data loads but it seems appropriate to consider it here also. One of the issues there is that 1G still feels like an awful lot for a minimum work size for each worker and it would mean we don't parallelize for relations less than that size. On a random VM on my personal server, an uncached 1G read takes over 10s. Cached it's less than half that, of course. This is all spinning rust (and only 7200 RPM at that) and there's a lot of other stuff going on but that still seems like too much of a chunk to give to one worker unless the overall data set to go through is really large. There's other issues in there too, of course, if we're dumping data in like this then we have to either deal with jagged relation files somehow or pad the file out to 1G, and that doesn't even get into the issues around how we'd have to redesign the interfaces for relation access and how this thinking is an utter violation of the modularity we currently have there. > > Yeah, I agree that's more typical. Robert's point that the master > > backend should participate is interesting but, as I recall, it was based > > on the idea that the master could finish faster than the worker- but if > > that's the case then we've planned it out wrong from the beginning. > > So, if the workers have been started but aren't keeping up, the master > should do nothing until they produce tuples rather than participating? > That doesn't seem right. Having the master jump in and start working could screw things up also though. Perhaps we need the master to start working as a fail-safe but not plan on having things go that way? Having more processes trying to do X doesn't always result in things getting better and the master needs to keep up with all the tuples being thrown at it from the workers. Thanks, Stephen
Amit, * Amit Kapila (amit.kapila16@gmail.com) wrote: > On Sun, Jan 11, 2015 at 9:09 AM, Robert Haas <robertmhaas@gmail.com> wrote: > > I don't think you should be "switching" queues. The tuples should be > > sent to the tuple queue, and errors and notices to the error queue. Agreed. > To achieve what you said (The tuples should be sent to the tuple > queue, and errors and notices to the error queue.), we need to > switch the queues. > The difficulty here is that once we set the queue (using > pq_redirect_to_shm_mq()) through which the communication has to > happen, it will use the same unless we change again the queue > using pq_redirect_to_shm_mq(). For example, assume we have > initially set error queue (using pq_redirect_to_shm_mq()) then to > send tuples, we need to call pq_redirect_to_shm_mq() to > set the tuple queue as the queue that needs to be used for communication > and again if error happens then we need to do the same for error > queue. > Do you have any other idea to achieve the same? I think what Robert's getting at here is that pq_redirect_to_shm_mq() might be fine for the normal data heading back, but we need something separate for errors and notices. Switching everything back and forth between the normal and error queues definitely doesn't sound right to me- they need to be independent. In other words, you need to be able to register a "normal data" queue and then you need to also register a "error/notice" queue and have errors and notices sent there directly. Going off of what I recall, can't this be done by having the callbacks which are registered for sending data back look at what they're being asked to send and then decide which queue it's appropriate for out of the set which have been registered so far? Thanks, Stephen
On 01/11/2015 11:27 AM, Stephen Frost wrote: > * Robert Haas (robertmhaas@gmail.com) wrote: >> On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote: >>> Yeah, if we come up with a plan for X workers and end up not being able >>> to spawn that many then I could see that being worth a warning or notice >>> or something. Not sure what EXPLAIN has to do anything with it.. >> >> That seems mighty odd to me. If there are 8 background worker >> processes available, and you allow each session to use at most 4, then >> when there are >2 sessions trying to do parallelism at the same time, >> they might not all get their workers. Emitting a notice for that >> seems like it would be awfully chatty. > > Yeah, agreed, it could get quite noisy. Did you have another thought > for how to address the concern raised? Specifically, that you might not > get as many workers as you thought you would? Wild idea: What about dealing with it as some sort of statistic - ie track some global counts in the stats collector or on a per-query base in pg_stat_activity and/or through pg_stat_statements? Not sure why it is that important to get it on a per-query base, imho it is simply a configuration limit we have set (similiar to work_mem or when switching to geqo) - we dont report "per query" through notice/warning there either (though the effect is kind visible in explain). Stefan
On Sat, Jan 10, 2015 at 11:14 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I don't think you should be "switching" queues. The tuples should be >> sent to the tuple queue, and errors and notices to the error queue. > To achieve what you said (The tuples should be sent to the tuple > queue, and errors and notices to the error queue.), we need to > switch the queues. > The difficulty here is that once we set the queue (using > pq_redirect_to_shm_mq()) through which the communication has to > happen, it will use the same unless we change again the queue > using pq_redirect_to_shm_mq(). For example, assume we have > initially set error queue (using pq_redirect_to_shm_mq()) then to > send tuples, we need to call pq_redirect_to_shm_mq() to > set the tuple queue as the queue that needs to be used for communication > and again if error happens then we need to do the same for error > queue. > Do you have any other idea to achieve the same? Yeah, you need two separate global variables pointing to shm_mq objects, one of which gets used by pqmq.c for errors and the other of which gets used by printtup.c for tuples. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sun, Jan 11, 2015 at 5:27 AM, Stephen Frost <sfrost@snowman.net> wrote: > * Robert Haas (robertmhaas@gmail.com) wrote: >> On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote: >> > Yeah, if we come up with a plan for X workers and end up not being able >> > to spawn that many then I could see that being worth a warning or notice >> > or something. Not sure what EXPLAIN has to do anything with it.. >> >> That seems mighty odd to me. If there are 8 background worker >> processes available, and you allow each session to use at most 4, then >> when there are >2 sessions trying to do parallelism at the same time, >> they might not all get their workers. Emitting a notice for that >> seems like it would be awfully chatty. > > Yeah, agreed, it could get quite noisy. Did you have another thought > for how to address the concern raised? Specifically, that you might not > get as many workers as you thought you would? I'm not sure why that's a condition in need of special reporting. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote: > So, for my 2c, I've long expected us to parallelize at the relation-file > level for these kinds of operations. This goes back to my other > thoughts on how we should be thinking about parallelizing inbound data > for bulk data loads but it seems appropriate to consider it here also. > One of the issues there is that 1G still feels like an awful lot for a > minimum work size for each worker and it would mean we don't parallelize > for relations less than that size. Yes, I think that's a killer objection. > [ .. ] and > how this thinking is an utter violation of the modularity we currently > have there. As is that. My thinking is more along the lines that we might need to issue explicit prefetch requests when doing a parallel sequential scan, to make up for any failure of the OS to do that for us. >> So, if the workers have been started but aren't keeping up, the master >> should do nothing until they produce tuples rather than participating? >> That doesn't seem right. > > Having the master jump in and start working could screw things up also > though. I don't think there's any reason why that should screw things up. There's no reason why the master's participation should look any different from one more worker. Look at my parallel_count code on the other thread to see what I mean: the master and all the workers are running the same code, and if fewer worker show up than expected, or run unduly slowly, it's easily tolerated. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sun, Jan 11, 2015 at 6:09 AM, Stephen Frost <sfrost@snowman.net> wrote: > I think what Robert's getting at here is that pq_redirect_to_shm_mq() > might be fine for the normal data heading back, but we need something > separate for errors and notices. Switching everything back and forth > between the normal and error queues definitely doesn't sound right to > me- they need to be independent. You've got that backwards. pq_redirect_to_shm_mq() handles errors and notices, but we need something separate for the tuple stream. > In other words, you need to be able to register a "normal data" queue > and then you need to also register a "error/notice" queue and have > errors and notices sent there directly. Going off of what I recall, > can't this be done by having the callbacks which are registered for > sending data back look at what they're being asked to send and then > decide which queue it's appropriate for out of the set which have been > registered so far? It's pretty simple, really. The functions that need to use the tuple queue are in printtup.c; those, and only those, need to be modified to write to the other queue. Or, possibly, we should pass the tuples around in their native format instead of translating them into binary form and then reconstituting them on the other end. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote:
> >> So, if the workers have been started but aren't keeping up, the master
> >> should do nothing until they produce tuples rather than participating?
> >> That doesn't seem right.
> >
> > Having the master jump in and start working could screw things up also
> > though.
>
> I don't think there's any reason why that should screw things up.
>
> On Sun, Jan 11, 2015 at 5:27 AM, Stephen Frost <sfrost@snowman.net> wrote:
> > * Robert Haas (robertmhaas@gmail.com) wrote:
> >> On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote:
> >> > Yeah, if we come up with a plan for X workers and end up not being able
> >> > to spawn that many then I could see that being worth a warning or notice
> >> > or something. Not sure what EXPLAIN has to do anything with it..
> >>
> >> That seems mighty odd to me. If there are 8 background worker
> >> processes available, and you allow each session to use at most 4, then
> >> when there are >2 sessions trying to do parallelism at the same time,
> >> they might not all get their workers. Emitting a notice for that
> >> seems like it would be awfully chatty.
> >
> > Yeah, agreed, it could get quite noisy. Did you have another thought
> > for how to address the concern raised? Specifically, that you might not
> > get as many workers as you thought you would?
>
> I'm not sure why that's a condition in need of special reporting.
>
So what should happen if no workers are available?
On 1/11/15 3:57 PM, Robert Haas wrote: > On Sun, Jan 11, 2015 at 5:27 AM, Stephen Frost <sfrost@snowman.net> wrote: >> * Robert Haas (robertmhaas@gmail.com) wrote: >>> On Thu, Jan 8, 2015 at 2:46 PM, Stephen Frost <sfrost@snowman.net> wrote: >>>> Yeah, if we come up with a plan for X workers and end up not being able >>>> to spawn that many then I could see that being worth a warning or notice >>>> or something. Not sure what EXPLAIN has to do anything with it.. >>> >>> That seems mighty odd to me. If there are 8 background worker >>> processes available, and you allow each session to use at most 4, then >>> when there are >2 sessions trying to do parallelism at the same time, >>> they might not all get their workers. Emitting a notice for that >>> seems like it would be awfully chatty. >> >> Yeah, agreed, it could get quite noisy. Did you have another thought >> for how to address the concern raised? Specifically, that you might not >> get as many workers as you thought you would? > > I'm not sure why that's a condition in need of special reporting. The case raised before (that I think is valid) is: what if you have a query that is massively parallel. You expect it toget 60 cores on the server and take 10 minutes. Instead it gets 10 and takes an hour (or worse, 1 and takes 10 hours). Maybe it's not worth dealing with that in the first version, but I expect it will come up very quickly. We better make surewe're not painting ourselves in a corner. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote:
> So, for my 2c, I've long expected us to parallelize at the relation-file
> level for these kinds of operations. This goes back to my other
> thoughts on how we should be thinking about parallelizing inbound data
> for bulk data loads but it seems appropriate to consider it here also.
> One of the issues there is that 1G still feels like an awful lot for a
> minimum work size for each worker and it would mean we don't parallelize
> for relations less than that size.
Yes, I think that's a killer objection.
On Sun, Jan 11, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote:
> So, for my 2c, I've long expected us to parallelize at the relation-file
> level for these kinds of operations. This goes back to my other
> thoughts on how we should be thinking about parallelizing inbound data
> for bulk data loads but it seems appropriate to consider it here also.
> One of the issues there is that 1G still feels like an awful lot for a
> minimum work size for each worker and it would mean we don't parallelize
> for relations less than that size.
Yes, I think that's a killer objection.One approach that I has worked well for me is to break big jobs into much smaller bite size tasks. Each task is small enough to complete quickly.We add the tasks to a task queue and spawn a generic worker pool which eats through the task queue items.This solves a lot of problems.- Small to medium jobs can be parallelized efficiently.- No need to split big jobs perfectly.- We don't get into a situation where we are waiting around for a worker to finish chugging through a huge task while the other workers sit idle.- Worker memory footprint is tiny so we can afford many of them.- Worker pool management is a well known problem.- Worker spawn time disappears as a cost factor.- The worker pool becomes a shared resource that can be managed and reported on and becomes considerably more predictable.
>
>
>
> On Sun, Jan 11, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:
>>
>> On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote:
>> > So, for my 2c, I've long expected us to parallelize at the relation-file
>> > level for these kinds of operations. This goes back to my other
>> > thoughts on how we should be thinking about parallelizing inbound data
>> > for bulk data loads but it seems appropriate to consider it here also.
>> > One of the issues there is that 1G still feels like an awful lot for a
>> > minimum work size for each worker and it would mean we don't parallelize
>> > for relations less than that size.
>>
>> Yes, I think that's a killer objection.
>
>
> One approach that I has worked well for me is to break big jobs into much smaller bite size tasks. Each task is small enough to complete quickly.
>
> We add the tasks to a task queue and spawn a generic worker pool which eats through the task queue items.
>
> This solves a lot of problems.
>
> - Small to medium jobs can be parallelized efficiently.
> - No need to split big jobs perfectly.
> - We don't get into a situation where we are waiting around for a worker to finish chugging through a huge task while the other workers sit idle.
> - Worker memory footprint is tiny so we can afford many of them.
> - Worker pool management is a well known problem.
> - Worker spawn time disappears as a cost factor.
> - The worker pool becomes a shared resource that can be managed and reported on and becomes considerably more predictable.
>
On Tue, Jan 13, 2015 at 4:55 PM, John Gorman <johngorman2@gmail.com> wrote:
>
>
>
> On Sun, Jan 11, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:
>>
>> On Sun, Jan 11, 2015 at 6:01 AM, Stephen Frost <sfrost@snowman.net> wrote:
>> > So, for my 2c, I've long expected us to parallelize at the relation-file
>> > level for these kinds of operations. This goes back to my other
>> > thoughts on how we should be thinking about parallelizing inbound data
>> > for bulk data loads but it seems appropriate to consider it here also.
>> > One of the issues there is that 1G still feels like an awful lot for a
>> > minimum work size for each worker and it would mean we don't parallelize
>> > for relations less than that size.
>>
>> Yes, I think that's a killer objection.
>
>
> One approach that I has worked well for me is to break big jobs into much smaller bite size tasks. Each task is small enough to complete quickly.
>Here we have to decide what should be the strategy and how mucheach worker should scan. As an example one of the the strategycould be if the table size is X MB and there are 8 workers, thendivide the work as X/8 MB for each worker (which I have currentlyused in patch) and another could be each worker does scan1 block at a time and then check some global structure to see whichnext block it needs to scan, according to me this could lead to randomscan. I have read that some other databases also divide the workbased on partitions or segments (size of segment is not very clear).
> We add the tasks to a task queue and spawn a generic worker pool which eats through the task queue items.
>
> This solves a lot of problems.
>
> - Small to medium jobs can be parallelized efficiently.
> - No need to split big jobs perfectly.
> - We don't get into a situation where we are waiting around for a worker to finish chugging through a huge task while the other workers sit idle.
> - Worker memory footprint is tiny so we can afford many of them.
> - Worker pool management is a well known problem.
> - Worker spawn time disappears as a cost factor.
> - The worker pool becomes a shared resource that can be managed and reported on and becomes considerably more predictable.
>Yeah, it is good idea to maintain shared worker pool, but it seemsto me that for initial version even if the workers are not shared,then also it is meaningful to make parallel sequential scan work.
--
Ashutosh Bapat
EnterpriseDB Corporation
The Postgres Database Company
On Tue, Jan 13, 2015 at 6:25 AM, John Gorman <johngorman2@gmail.com> wrote: > One approach that I has worked well for me is to break big jobs into much > smaller bite size tasks. Each task is small enough to complete quickly. > > We add the tasks to a task queue and spawn a generic worker pool which eats > through the task queue items. > > This solves a lot of problems. > > - Small to medium jobs can be parallelized efficiently. > - No need to split big jobs perfectly. > - We don't get into a situation where we are waiting around for a worker to > finish chugging through a huge task while the other workers sit idle. > - Worker memory footprint is tiny so we can afford many of them. > - Worker pool management is a well known problem. > - Worker spawn time disappears as a cost factor. > - The worker pool becomes a shared resource that can be managed and reported > on and becomes considerably more predictable. I think this is a good idea, but for now I would like to keep our goals somewhat more modest: let's see if we can get parallel sequential scan, and only parallel sequential scan, working and committed. Ultimately, I think we may need something like what you're talking about, because if you have a query with three or six or twelve different parallelizable operations in it, you want the available CPU resources to switch between those as their respective needs may dictate. You certainly don't want to spawn a separate pool of workers for each scan. But I think getting that all working in the first version is probably harder than what we should attempt. We have a bunch of problems to solve here just around parallel sequential scan and the parallel mode infrastructure: heavyweight locking, prefetching, the cost model, and so on. Trying to add to that all of the problems that might attend on a generic task queueing infrastructure fills me with no small amount of fear. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 1/13/15 9:42 PM, Amit Kapila wrote: > As an example one of the the strategy > could be if the table size is X MB and there are 8 workers, then > divide the work as X/8 MB for each worker (which I have currently > used in patch) and another could be each worker does scan > 1 block at a time and then check some global structure to see which > next block it needs to scan, according to me this could lead to random > scan. I have read that some other databases also divide the work > based on partitions or segments (size of segment is not very clear). Long-term I think we'll want a mix between the two approaches. Simply doing something like blkno % num_workers is going tocause imbalances, but trying to do this on a per-block basis seems like too much overhead. Also long-term, I think we also need to look at a more specialized version of parallelism at the IO layer. For example, duringan index scan you'd really like to get IO requests for heap blocks started in the background while the backend is focusedon the mechanics of the index scan itself. While this could be done with the stuff Robert has written I have to wonderif it'd be a lot more efficient to use fadvise or AIO. Or perhaps it would just be better to deal with an entire indexpage (remembering TIDs) and then hit the heap. But I agree with Robert; there's a lot yet to be done just to get *any* kind of parallel execution working before we startthinking about how to optimize it. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
>
> On Wed, Jan 14, 2015 at 9:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> Here we have to decide what should be the strategy and how much
>> each worker should scan. As an example one of the the strategy
>> could be if the table size is X MB and there are 8 workers, then
>> divide the work as X/8 MB for each worker (which I have currently
>> used in patch) and another could be each worker does scan
>> 1 block at a time and then check some global structure to see which
>> next block it needs to scan, according to me this could lead to random
>> scan. I have read that some other databases also divide the work
>> based on partitions or segments (size of segment is not very clear).
>
>
> A block can contain useful tuples, i.e tuples which are visible and fulfil the quals + useless tuples i.e. tuples which are dead, invisible or that do not fulfil the quals. Depending upon the contents of these blocks, esp. the ratio of (useful tuples)/(unuseful tuples), even though we divide the relation into equal sized runs, each worker may take different time. So, instead of dividing the relation into number of run = number of workers, it might be better to divide them into fixed sized runs with size < (total number of blocks/ number of workers), and let a worker pick up a run after it finishes with the previous one. The smaller the size of runs the better load balancing but higher cost of starting with the run. So, we have to strike a balance.
>
I think your suggestion is good and it somewhat falls inline
>
> On Sat, Jan 10, 2015 at 11:14 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> I don't think you should be "switching" queues. The tuples should be
> >> sent to the tuple queue, and errors and notices to the error queue.
> > To achieve what you said (The tuples should be sent to the tuple
> > queue, and errors and notices to the error queue.), we need to
> > switch the queues.
> > The difficulty here is that once we set the queue (using
> > pq_redirect_to_shm_mq()) through which the communication has to
> > happen, it will use the same unless we change again the queue
> > using pq_redirect_to_shm_mq(). For example, assume we have
> > initially set error queue (using pq_redirect_to_shm_mq()) then to
> > send tuples, we need to call pq_redirect_to_shm_mq() to
> > set the tuple queue as the queue that needs to be used for communication
> > and again if error happens then we need to do the same for error
> > queue.
> > Do you have any other idea to achieve the same?
>
> Yeah, you need two separate global variables pointing to shm_mq
> objects, one of which gets used by pqmq.c for errors and the other of
> which gets used by printtup.c for tuples.
>
Okay, I will try to change the way as suggested without doing
On Wed, Jan 14, 2015 at 9:00 PM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote: > Simply doing > something like blkno % num_workers is going to cause imbalances, Yes. > but trying > to do this on a per-block basis seems like too much overhead. ...but no. Or at least, I doubt it. The cost of handing out blocks one at a time is that, for each block, a worker's got to grab a spinlock, increment and record the block number counter, and release the spinlock. Or, use an atomic add. Now, it's true that spinlock cycles and atomic ops can have sometimes impose severe overhead, but you have to look at it as a percentage of the overall work being done. In this case, the backend has to read, pin, and lock the page and process every tuple on the page. Processing every tuple on the page may involve de-TOASTing the tuple (leading to many more page accesses), or evaluating a complex expression, or hitting CLOG to check visibility, but even if it doesn't, I think the amount of work that it takes to process all the tuples on the page will be far larger than the cost of one atomic increment operation per block. As mentioned downthread, a far bigger consideration is the I/O pattern we create. A sequential scan is so-called because it reads the relation sequentially. If we destroy that property, we will be more than slightly sad. It might be OK to do sequential scans of, say, each 1GB segment separately, but I'm pretty sure it would be a real bad idea to read 8kB at a time at blocks 0, 64, 128, 1, 65, 129, ... What I'm thinking about is that we might have something like this: struct this_lives_in_dynamic_shared_memory { BlockNumber last_block; Size prefetch_distance; Size prefetch_increment; slock_t mutex; BlockNumber next_prefetch_block; BlockNumber next_scan_block; }; Each worker takes the mutex and checks whether next_prefetch_block - next_scan_block < prefetch_distance and also whether next_prefetch_block < last_block. If both are true, it prefetches some number of additional blocks, as specified by prefetch_increment. Otherwise, it increments next_scan_block and scans the block corresponding to the old value. So in this way, the prefetching runs ahead of the scan by a configurable amount (prefetch_distance), which should be chosen so that the prefetches have time to compete before the scan actually reaches those blocks. Right now, of course, we rely on the operating system to prefetch for sequential scans, but I have a strong hunch that may not work on all systems if there are multiple processes doing the reads. Now, what of other strategies like dividing up the relation into 1GB chunks and reading each one in a separate process? We could certainly DO that, but what advantage does it have over this? The only benefit I can see is that you avoid accessing a data structure of the type shown above for every block, but that only matters if that cost is material, and I tend to think it won't be. On the flip side, it means that the granularity for dividing up work between processes is now very coarse - when there are less than 6GB of data left in a relation, at most 6 processes can work on it. That might be OK if the data is being read in from disk anyway, but it's certainly not the best we can do when the data is in memory. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> As mentioned downthread, a far bigger consideration is the I/O pattern
> we create. A sequential scan is so-called because it reads the
> relation sequentially. If we destroy that property, we will be more
> than slightly sad. It might be OK to do sequential scans of, say,
> each 1GB segment separately, but I'm pretty sure it would be a real
> bad idea to read 8kB at a time at blocks 0, 64, 128, 1, 65, 129, ...
>
> What I'm thinking about is that we might have something like this:
>
> struct this_lives_in_dynamic_shared_memory
> {
> BlockNumber last_block;
> Size prefetch_distance;
> Size prefetch_increment;
> slock_t mutex;
> BlockNumber next_prefetch_block;
> BlockNumber next_scan_block;
> };
>
> Each worker takes the mutex and checks whether next_prefetch_block -
> next_scan_block < prefetch_distance and also whether
> next_prefetch_block < last_block. If both are true, it prefetches
> some number of additional blocks, as specified by prefetch_increment.
> Otherwise, it increments next_scan_block and scans the block
> corresponding to the old value.
>
On Fri, Jan 16, 2015 at 11:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Jan 16, 2015 at 11:49 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> As mentioned downthread, a far bigger consideration is the I/O pattern >> we create. A sequential scan is so-called because it reads the >> relation sequentially. If we destroy that property, we will be more >> than slightly sad. It might be OK to do sequential scans of, say, >> each 1GB segment separately, but I'm pretty sure it would be a real >> bad idea to read 8kB at a time at blocks 0, 64, 128, 1, 65, 129, ... >> >> What I'm thinking about is that we might have something like this: >> >> struct this_lives_in_dynamic_shared_memory >> { >> BlockNumber last_block; >> Size prefetch_distance; >> Size prefetch_increment; >> slock_t mutex; >> BlockNumber next_prefetch_block; >> BlockNumber next_scan_block; >> }; >> >> Each worker takes the mutex and checks whether next_prefetch_block - >> next_scan_block < prefetch_distance and also whether >> next_prefetch_block < last_block. If both are true, it prefetches >> some number of additional blocks, as specified by prefetch_increment. >> Otherwise, it increments next_scan_block and scans the block >> corresponding to the old value. > > Assuming we will increment next_prefetch_block only after prefetching > blocks (equivalent to prefetch_increment), won't 2 workers can > simultaneously see the same value for next_prefetch_block and try to > perform prefetch for same blocks? The idea is that you can only examine and modify next_prefetch_block or next_scan_block while holding the mutex. > What will be value of prefetch_increment? > Will it be equal to prefetch_distance or prefetch_distance/2 or > prefetch_distance/4 or .. or will it be totally unrelated to > prefetch_distance? I dunno, that might take some experimentation. prefetch_distance/2 doesn't sound stupid. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> On Fri, Jan 16, 2015 at 11:27 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Assuming we will increment next_prefetch_block only after prefetching
> > blocks (equivalent to prefetch_increment), won't 2 workers can
> > simultaneously see the same value for next_prefetch_block and try to
> > perform prefetch for same blocks?
>
> The idea is that you can only examine and modify next_prefetch_block
> or next_scan_block while holding the mutex.
>
> > What will be value of prefetch_increment?
> > Will it be equal to prefetch_distance or prefetch_distance/2 or
> > prefetch_distance/4 or .. or will it be totally unrelated to
> > prefetch_distance?
>
> I dunno, that might take some experimentation. prefetch_distance/2
> doesn't sound stupid.
>
On Mon, Jan 19, 2015 at 2:24 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Okay, I think I got the idea what you want to achieve via > prefetching. So assuming prefetch_distance = 100 and > prefetch_increment = 50 (prefetch_distance /2), it seems to me > that as soon as there are less than 100 blocks in prefetch quota, > it will fetch next 50 blocks which means the system will be always > approximately 50 blocks ahead, that will ensure that in this algorithm > it will always perform sequential scan, however eventually this is turning > to be a system where one worker is reading from disk and then other > workers are reading from OS buffers to shared buffers and then getting > the tuple. In this approach only one downside I can see and that is > there could be times during execution where some/all workers will have > to wait on the worker doing prefetching, however I think we should try > this approach and see how it works. Right. We probably want to make prefetch_distance a GUC. After all, we currently rely on the operating system for prefetching, and the operating system has a setting for this, at least on Linux (blockdev --getra). It's possible, however, that we don't need this at all, because the OS might be smart enough to figure it out for us. It's probably worth testing, though. > Another thing is that I think prefetching is not supported on all platforms > (Windows) and for such systems as per above algorithm we need to > rely on block-by-block method. Well, I think we should try to set up a test to see if this is hurting us. First, do a sequential-scan of a related too big at least twice as large as RAM. Then, do a parallel sequential scan of the same relation with 2 workers. Repeat these in alternation several times. If the operating system is accomplishing meaningful readahead, and the parallel sequential scan is breaking it, then since the test is I/O-bound I would expect to see the parallel scan actually being slower than the normal way. Or perhaps there is some other test that would be better (ideas welcome) but the point is we may need something like this, but we should try to figure out whether we need it before spending too much time on it. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > Yeah, you need two separate global variables pointing to shm_mq
> > objects, one of which gets used by pqmq.c for errors and the other of
> > which gets used by printtup.c for tuples.
> >
>
> Okay, I will try to change the way as suggested without doing
> switching, but this way we need to do it separately for 'T', 'D', and
> 'C' messages.
>
I have taken care of integrating the parallel sequence scan with the
Attachment
On Thu, Jan 15, 2015 at 6:57 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > Yeah, you need two separate global variables pointing to shm_mq
> > objects, one of which gets used by pqmq.c for errors and the other of
> > which gets used by printtup.c for tuples.
> >
>
> Okay, I will try to change the way as suggested without doing
> switching, but this way we need to do it separately for 'T', 'D', and
> 'C' messages.
>
I have taken care of integrating the parallel sequence scan with thelatest patch posted (parallel-mode-v1.patch) by Robert at belowlocation:Changes in this version-----------------------------------------------1. As mentioned previously, I have exposed one parameterParallelWorkerNumber as used in parallel-mode patch.2. Enabled tuple queue to be used for passing tuples fromworker backend to master backend along with error queueas per suggestion by Robert in the mail above.3. Involved master backend to scan the heap directly whentuples are not available in any shared memory tuple queue.4. Introduced 3 new parameters (cpu_tuple_comm_cost,parallel_setup_cost, parallel_startup_cost) for deciding the costof parallel plan. Currently, I have kept the default values forparallel_setup_cost and parallel_startup_cost as 0.0, as thoserequire some experiments.5. Fixed some issues (related to memory increase as reportedupthread by Thom Brown and general feature issues found duringtest)Note - I have yet to handle the new node types introduced at someof the places and need to verify prepared queries and some otherthings, however I think it will be good if I can get some feedbackat current stage.
thom@swift:~/Development/postgresql$ patch -p1 < ~/Downloads/parallel_seqscan_v4.patch
patching file src/backend/access/Makefile
patching file src/backend/access/common/printtup.c
patching file src/backend/access/shmmq/Makefile
patching file src/backend/access/shmmq/shmmqam.c
patching file src/backend/commands/explain.c
Hunk #1 succeeded at 721 (offset 8 lines).
Hunk #2 succeeded at 918 (offset 8 lines).
Hunk #3 succeeded at 1070 (offset 8 lines).
Hunk #4 succeeded at 1337 (offset 8 lines).
Hunk #5 succeeded at 2239 (offset 83 lines).
patching file src/backend/executor/Makefile
patching file src/backend/executor/execProcnode.c
patching file src/backend/executor/execScan.c
patching file src/backend/executor/execTuples.c
patching file src/backend/executor/nodeParallelSeqscan.c
patching file src/backend/executor/nodeSeqscan.c
patching file src/backend/libpq/pqmq.c
Hunk #1 succeeded at 23 with fuzz 2 (offset -3 lines).
Hunk #2 FAILED at 63.
Hunk #3 succeeded at 132 (offset -31 lines).
1 out of 3 hunks FAILED -- saving rejects to file src/backend/libpq/pqmq.c.rej
patching file src/backend/optimizer/path/Makefile
patching file src/backend/optimizer/path/allpaths.c
patching file src/backend/optimizer/path/costsize.c
patching file src/backend/optimizer/path/parallelpath.c
patching file src/backend/optimizer/plan/createplan.c
patching file src/backend/optimizer/plan/planner.c
patching file src/backend/optimizer/plan/setrefs.c
patching file src/backend/optimizer/util/pathnode.c
patching file src/backend/postmaster/Makefile
patching file src/backend/postmaster/backendworker.c
patching file src/backend/postmaster/postmaster.c
patching file src/backend/tcop/dest.c
patching file src/backend/tcop/postgres.c
Hunk #1 succeeded at 54 (offset -1 lines).
Hunk #2 succeeded at 1132 (offset -1 lines).
patching file src/backend/utils/misc/guc.c
patching file src/backend/utils/misc/postgresql.conf.sample
can't find file to patch at input line 2105
Perhaps you used the wrong -p or --strip option?
The text leading up to this was:
--------------------------
|diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
|index 761ba1f..00ad468 100644
|--- a/src/include/access/parallel.h
|+++ b/src/include/access/parallel.h
--------------------------
File to patch:
>
> On 20 January 2015 at 14:29, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> On Thu, Jan 15, 2015 at 6:57 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>> > On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>> > >
>> > > Yeah, you need two separate global variables pointing to shm_mq
>> > > objects, one of which gets used by pqmq.c for errors and the other of
>> > > which gets used by printtup.c for tuples.
>> > >
>> >
>> > Okay, I will try to change the way as suggested without doing
>> > switching, but this way we need to do it separately for 'T', 'D', and
>> > 'C' messages.
>> >
>>
>> I have taken care of integrating the parallel sequence scan with the
>> latest patch posted (parallel-mode-v1.patch) by Robert at below
>> location:
>> http://www.postgresql.org/message-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
>>
>> Changes in this version
>> -----------------------------------------------
>> 1. As mentioned previously, I have exposed one parameter
>> ParallelWorkerNumber as used in parallel-mode patch.
>> 2. Enabled tuple queue to be used for passing tuples from
>> worker backend to master backend along with error queue
>> as per suggestion by Robert in the mail above.
>> 3. Involved master backend to scan the heap directly when
>> tuples are not available in any shared memory tuple queue.
>> 4. Introduced 3 new parameters (cpu_tuple_comm_cost,
>> parallel_setup_cost, parallel_startup_cost) for deciding the cost
>> of parallel plan. Currently, I have kept the default values for
>> parallel_setup_cost and parallel_startup_cost as 0.0, as those
>> require some experiments.
>> 5. Fixed some issues (related to memory increase as reported
>> upthread by Thom Brown and general feature issues found during
>> test)
>>
>> Note - I have yet to handle the new node types introduced at some
>> of the places and need to verify prepared queries and some other
>> things, however I think it will be good if I can get some feedback
>> at current stage.
>
>
> Which commit is this based against? I'm getting errors with the latest master:
>
On Tue, Jan 20, 2015 at 9:43 PM, Thom Brown <thom@linux.com> wrote:
>
> On 20 January 2015 at 14:29, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> On Thu, Jan 15, 2015 at 6:57 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>> > On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>> > >
>> > > Yeah, you need two separate global variables pointing to shm_mq
>> > > objects, one of which gets used by pqmq.c for errors and the other of
>> > > which gets used by printtup.c for tuples.
>> > >
>> >
>> > Okay, I will try to change the way as suggested without doing
>> > switching, but this way we need to do it separately for 'T', 'D', and
>> > 'C' messages.
>> >
>>
>> I have taken care of integrating the parallel sequence scan with the
>> latest patch posted (parallel-mode-v1.patch) by Robert at below
>> location:
>> http://www.postgresql.org/message-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
>>
>> Changes in this version
>> -----------------------------------------------
>> 1. As mentioned previously, I have exposed one parameter
>> ParallelWorkerNumber as used in parallel-mode patch.
>> 2. Enabled tuple queue to be used for passing tuples from
>> worker backend to master backend along with error queue
>> as per suggestion by Robert in the mail above.
>> 3. Involved master backend to scan the heap directly when
>> tuples are not available in any shared memory tuple queue.
>> 4. Introduced 3 new parameters (cpu_tuple_comm_cost,
>> parallel_setup_cost, parallel_startup_cost) for deciding the cost
>> of parallel plan. Currently, I have kept the default values for
>> parallel_setup_cost and parallel_startup_cost as 0.0, as those
>> require some experiments.
>> 5. Fixed some issues (related to memory increase as reported
>> upthread by Thom Brown and general feature issues found during
>> test)
>>
>> Note - I have yet to handle the new node types introduced at some
>> of the places and need to verify prepared queries and some other
>> things, however I think it will be good if I can get some feedback
>> at current stage.
>
>
> Which commit is this based against? I'm getting errors with the latest master:
>It seems to me that you have not applied parallel-mode patchbefore applying this patch, can you try once again by first applyingthe patch posted by Robert at below link:commit-id used for this patch - 0b49642
Thanks.
On Thu, Jan 15, 2015 at 6:57 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > Yeah, you need two separate global variables pointing to shm_mq
> > objects, one of which gets used by pqmq.c for errors and the other of
> > which gets used by printtup.c for tuples.
> >
>
> Okay, I will try to change the way as suggested without doing
> switching, but this way we need to do it separately for 'T', 'D', and
> 'C' messages.
>
I have taken care of integrating the parallel sequence scan with thelatest patch posted (parallel-mode-v1.patch) by Robert at belowlocation:Changes in this version-----------------------------------------------1. As mentioned previously, I have exposed one parameterParallelWorkerNumber as used in parallel-mode patch.2. Enabled tuple queue to be used for passing tuples fromworker backend to master backend along with error queueas per suggestion by Robert in the mail above.3. Involved master backend to scan the heap directly whentuples are not available in any shared memory tuple queue.4. Introduced 3 new parameters (cpu_tuple_comm_cost,parallel_setup_cost, parallel_startup_cost) for deciding the costof parallel plan. Currently, I have kept the default values forparallel_setup_cost and parallel_startup_cost as 0.0, as thoserequire some experiments.5. Fixed some issues (related to memory increase as reportedupthread by Thom Brown and general feature issues found duringtest)Note - I have yet to handle the new node types introduced at someof the places and need to verify prepared queries and some otherthings, however I think it will be good if I can get some feedbackat current stage.
➤ psql://thom@[local]:5488/pgbench
# set parallel_seqscan_degree = 8;
SET
Time: 0.248 ms
➤ psql://thom@[local]:5488/pgbench
# explain select c1 from t1;
QUERY PLAN
--------------------------------------------------------------
Parallel Seq Scan on t1 (cost=0.00..21.22 rows=100 width=4)
Number of Workers: 8
Number of Blocks Per Worker: 11
(3 rows)
Time: 0.322 ms
# explain analyse select c1 from t1;
QUERY PLAN
-----------------------------------------------------------------------------------------------------------
Parallel Seq Scan on t1 (cost=0.00..21.22 rows=100 width=4) (actual time=0.024..13.468 rows=100 loops=1)
Number of Workers: 8
Number of Blocks Per Worker: 11
Planning time: 0.040 ms
Execution time: 13.862 ms
(5 rows)
Time: 14.188 ms
➤ psql://thom@[local]:5488/pgbench
# set parallel_seqscan_degree = 10;
SET
Time: 0.219 ms
➤ psql://thom@[local]:5488/pgbench
# explain select c1 from t1;
QUERY PLAN
--------------------------------------------------------------
Parallel Seq Scan on t1 (cost=0.00..19.18 rows=100 width=4)
Number of Workers: 10
Number of Blocks Per Worker: 9
(3 rows)
Time: 0.375 ms
➤ psql://thom@[local]:5488/pgbench
# explain analyse select c1 from t1;
# explain (analyse, buffers, timing) select distinct bid from pgbench_accounts;
QUERY PLAN
------------------------------------------------------------------------------------------------------------------------------------------------
HashAggregate (cost=1400411.11..1400412.11 rows=100 width=4) (actual time=8504.333..8504.335 rows=13 loops=1)
Group Key: bid
Buffers: shared hit=32 read=18183
-> Parallel Seq Scan on pgbench_accounts (cost=0.00..1375411.11 rows=10000000 width=4) (actual time=0.054..7183.494 rows=10000000 loops=1)
Number of Workers: 8
Number of Blocks Per Worker: 18215
Buffers: shared hit=32 read=18183
Planning time: 0.058 ms
Execution time: 8876.967 ms
(9 rows)
Time: 8877.366 ms
On 1/19/15 7:20 AM, Robert Haas wrote: >> >Another thing is that I think prefetching is not supported on all platforms >> >(Windows) and for such systems as per above algorithm we need to >> >rely on block-by-block method. > Well, I think we should try to set up a test to see if this is hurting > us. First, do a sequential-scan of a related too big at least twice > as large as RAM. Then, do a parallel sequential scan of the same > relation with 2 workers. Repeat these in alternation several times. > If the operating system is accomplishing meaningful readahead, and the > parallel sequential scan is breaking it, then since the test is > I/O-bound I would expect to see the parallel scan actually being > slower than the normal way. > > Or perhaps there is some other test that would be better (ideas > welcome) but the point is we may need something like this, but we > should try to figure out whether we need it before spending too much > time on it. I'm guessing that not all supported platforms have prefetching that actually helps us... but it would be good to actuallyknow if that's the case. Where I think this gets a lot more interesting is if we could apply this to an index scan. My thought is that would resultin one worker mostly being responsible for advancing the index scan itself while the other workers were issuing (andwaiting on) heap IO. So even if this doesn't turn out to be a win for seqscan, there's other places we might well wantto use it. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
>
> On 20 January 2015 at 14:29, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> Note - I have yet to handle the new node types introduced at some
>> of the places and need to verify prepared queries and some other
>> things, however I think it will be good if I can get some feedback
>> at current stage.
>
>
> I'm getting an issue:
>
>
>
> # set parallel_seqscan_degree = 10;
> SET
> Time: 0.219 ms
>
> ➤ psql://thom@[local]:5488/pgbench
>
>
> ➤ psql://thom@[local]:5488/pgbench
>
> # explain analyse select c1 from t1;
>
>
> So setting parallel_seqscan_degree above max_worker_processes causes the CPU to max out, and the query never returns, or at least not after waiting 2 minutes. Shouldn't it have a ceiling of max_worker_processes?
>
> The original test I performed where I was getting OOM errors now appears to be fine:
>
On 20-01-2015 PM 11:29, Amit Kapila wrote: > > I have taken care of integrating the parallel sequence scan with the > latest patch posted (parallel-mode-v1.patch) by Robert at below > location: > http://www.postgresql.org/message-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com > > Changes in this version > ----------------------------------------------- > 1. As mentioned previously, I have exposed one parameter > ParallelWorkerNumber as used in parallel-mode patch. > 2. Enabled tuple queue to be used for passing tuples from > worker backend to master backend along with error queue > as per suggestion by Robert in the mail above. > 3. Involved master backend to scan the heap directly when > tuples are not available in any shared memory tuple queue. > 4. Introduced 3 new parameters (cpu_tuple_comm_cost, > parallel_setup_cost, parallel_startup_cost) for deciding the cost > of parallel plan. Currently, I have kept the default values for > parallel_setup_cost and parallel_startup_cost as 0.0, as those > require some experiments. > 5. Fixed some issues (related to memory increase as reported > upthread by Thom Brown and general feature issues found during > test) > > Note - I have yet to handle the new node types introduced at some > of the places and need to verify prepared queries and some other > things, however I think it will be good if I can get some feedback > at current stage. > I got an assertion failure: In src/backend/executor/execTuples.c: ExecStoreTuple() /* passing shouldFree=true for a tuple on a disk page is not sane */ Assert(BufferIsValid(buffer) ? (!shouldFree) : true); when called from: In src/backend/executor/nodeParallelSeqscan.c: ParallelSeqNext() I think something like the following would be necessary (reading from comments in the code): --- a/src/backend/executor/nodeParallelSeqscan.c +++ b/src/backend/executor/nodeParallelSeqscan.c @@ -85,7 +85,7 @@ ParallelSeqNext(ParallelSeqScanState *node) if (tuple) ExecStoreTuple(tuple, slot, - scandesc->rs_cbuf, + fromheap ? scandesc->rs_cbuf : InvalidBuffer, !fromheap); After fixing this, the assertion failure seems to be gone though I observed the blocked (CPU maxed out) state as reported elsewhere by Thom Brown. What I was doing: CREATE TABLE test(a) AS SELECT generate_series(1, 10000000); postgres=# SHOW max_worker_processes;max_worker_processes ----------------------8 (1 row) postgres=# SET seq_page_cost TO 100; SET postgres=# SET parallel_seqscan_degree TO 4; SET postgres=# EXPLAIN SELECT * FROM test; QUERY PLAN -------------------------------------------------------------------------Parallel Seq Scan on test (cost=0.00..1801071.27rows=8981483 width=4) Number of Workers: 4 Number of Blocks Per Worker: 8849 (3 rows) Though, EXPLAIN ANALYZE caused the thing. Thanks, Amit
>
> On 20-01-2015 PM 11:29, Amit Kapila wrote:
> > Note - I have yet to handle the new node types introduced at some
> > of the places and need to verify prepared queries and some other
> > things, however I think it will be good if I can get some feedback
> > at current stage.
> >
>
> I got an assertion failure:
>
> In src/backend/executor/execTuples.c: ExecStoreTuple()
>
> /* passing shouldFree=true for a tuple on a disk page is not sane */
> Assert(BufferIsValid(buffer) ? (!shouldFree) : true);
>
> After fixing this, the assertion failure seems to be gone though I
> observed the blocked (CPU maxed out) state as reported elsewhere by Thom
> Brown.
>
On Wednesday, January 21, 2015, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Wed, Jan 21, 2015 at 12:47 PM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote:
>
> On 20-01-2015 PM 11:29, Amit Kapila wrote:
> > Note - I have yet to handle the new node types introduced at some
> > of the places and need to verify prepared queries and some other
> > things, however I think it will be good if I can get some feedback
> > at current stage.
> >
>
> I got an assertion failure:
>
> In src/backend/executor/execTuples.c: ExecStoreTuple()
>
> /* passing shouldFree=true for a tuple on a disk page is not sane */
> Assert(BufferIsValid(buffer) ? (!shouldFree) : true);
>Good Catch!The reason is that while master backend is scanning from a heappage, if it finds another tuple/tuples's from shared memory messagequeue it will process those tuples first and in such a scenario, the scandescriptor will still have reference to buffer which it is using from scanningfrom heap. Your proposed fix will work.
> After fixing this, the assertion failure seems to be gone though I
> observed the blocked (CPU maxed out) state as reported elsewhere by Thom
> Brown.
>Does it happen only when parallel_seqscan_degree > max_worker_processes?
> On Wednesday, January 21, 2015, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>>
>> Does it happen only when parallel_seqscan_degree > max_worker_processes?
>
>
> I have max_worker_processes set to the default of 8 while parallel_seqscan_degree is 4. So, this may be a case different from Thom's.
>
I think this is due to reason that memory for forming
Attachment
> On Wed, Jan 21, 2015 at 4:31 PM, Amit Langote <amitlangote09@gmail.com> > wrote: > > On Wednesday, January 21, 2015, Amit Kapila <amit.kapila16@gmail.com> > wrote: > >> > >> > >> Does it happen only when parallel_seqscan_degree > max_worker_processes? > > > > > > I have max_worker_processes set to the default of 8 while > parallel_seqscan_degree is 4. So, this may be a case different from Thom's. > > > > I think this is due to reason that memory for forming tuple in master backend > is retained for longer time which is causing this statement to take much > longer time than required. I have fixed the other issue as well reported > by you in attached patch. > > I think this patch is still not completely ready for general purpose testing, > however it could be helpful if we can run some tests to see in what kind > of scenario's it gives benefit like in the test you are doing if rather > than increasing seq_page_cost, you should add an expensive WHERE condition > so that it should automatically select parallel plan. I think it is better > to change one of the new parameter's (parallel_setup_cost, > parallel_startup_cost and cpu_tuple_comm_cost) if you want your statement > to use parallel plan, like in your example if you would have reduced > cpu_tuple_comm_cost, it would have selected parallel plan, that way we can > get some feedback about what should be the appropriate default values for > the newly added parameters. I am already planing to do some tests in that > regard, however if I get some feedback from other's that would be helpful. > (Please point out me if my understanding is incorrect.) What happen if dynamic background worker process tries to reference temporary tables? Because buffer of temporary table blocks are allocated on private address space, its recent status is not visible to other process unless it is not flushed to the storage every time. Do we need to prohibit create_parallelscan_paths() to generate a path when target relation is temporary one? Thanks, -- NEC OSS Promotion Center / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
>
> (Please point out me if my understanding is incorrect.)
>
> tables? Because buffer of temporary table blocks are allocated on private
> address space, its recent status is not visible to other process unless it is
> not flushed to the storage every time.
>
> Do we need to prohibit create_parallelscan_paths() to generate a path when
> target relation is temporary one?
>
Yes, we need to prohibit parallel scans on temporary relations. Will fix.
On 21-01-2015 PM 09:43, Amit Kapila wrote: > On Wed, Jan 21, 2015 at 4:31 PM, Amit Langote <amitlangote09@gmail.com> > wrote: >> On Wednesday, January 21, 2015, Amit Kapila <amit.kapila16@gmail.com> > wrote: >>> >>> >>> Does it happen only when parallel_seqscan_degree > max_worker_processes? >> >> >> I have max_worker_processes set to the default of 8 while > parallel_seqscan_degree is 4. So, this may be a case different from Thom's. >> > > I think this is due to reason that memory for forming > tuple in master backend is retained for longer time which > is causing this statement to take much longer time than > required. I have fixed the other issue as well reported by > you in attached patch. > Thanks for fixing. > I think this patch is still not completely ready for general > purpose testing, however it could be helpful if we can run > some tests to see in what kind of scenario's it gives benefit > like in the test you are doing if rather than increasing > seq_page_cost, you should add an expensive WHERE condition > so that it should automatically select parallel plan. I think it is better > to change one of the new parameter's (parallel_setup_cost, > parallel_startup_cost and cpu_tuple_comm_cost) if you want > your statement to use parallel plan, like in your example if > you would have reduced cpu_tuple_comm_cost, it would have > selected parallel plan, that way we can get some feedback about > what should be the appropriate default values for the newly added > parameters. I am already planing to do some tests in that regard, > however if I get some feedback from other's that would be helpful. > > Perhaps you are aware or you've postponed working on it, but I see that a plan executing in a worker does not know about instrumentation. It results in the EXPLAIN ANALYZE showing incorrect figures. For example compare the normal seqscan and parallel seqscan below: postgres=# EXPLAIN ANALYZE SELECT * FROM test WHERE sqrt(a) < 3456 AND md5(a::text) LIKE 'ac%'; QUERY PLAN ---------------------------------------------------------------------------------------------------------------Seq Scan ontest (cost=0.00..310228.52 rows=16120 width=4) (actual time=0.497..17062.436 rows=39028 loops=1) Filter: ((sqrt((a)::double precision) < 3456::double precision) AND (md5((a)::text) ~~ 'ac%'::text)) Rows Removed by Filter: 9960972Planning time: 0.206 msExecution time: 17378.413 ms (5 rows) postgres=# EXPLAIN ANALYZE SELECT * FROM test WHERE sqrt(a) < 3456 AND md5(a::text) LIKE 'ac%'; QUERY PLAN -----------------------------------------------------------------------------------------------------------------------Parallel SeqScan on test (cost=0.00..255486.08 rows=16120 width=4) (actual time=7.329..4906.981 rows=39028 loops=1) Filter: ((sqrt((a)::double precision) < 3456::double precision) AND (md5((a)::text) ~~ 'ac%'::text)) Rows Removed by Filter: 1992710 Number of Workers: 4 Number of Blocks Per Worker: 8849Planningtime: 0.137 msExecution time: 6077.782 ms (7 rows) Note the "Rows Removed by Filter". I guess the difference may be because, all the rows filtered by workers were not accounted for. I'm not quite sure, but since exec_worker_stmt goes the Portal way, QueryDesc.instrument_options remains unset and hence no instrumentation opportunities in a worker backend. One option may be to pass instrument_options down to worker_stmt? By the way, 17s and 6s compare really well in favor of parallel seqscan above, :) Thanks, Amit
>
> On 21-01-2015 PM 09:43, Amit Kapila wrote:
> > On Wed, Jan 21, 2015 at 4:31 PM, Amit Langote <amitlangote09@gmail.com>
> > wrote:
> >> On Wednesday, January 21, 2015, Amit Kapila <amit.kapila16@gmail.com>
> > wrote:
> >>>
> >>>
> >>> Does it happen only when parallel_seqscan_degree > max_worker_processes?
> >>
> >>
> >> I have max_worker_processes set to the default of 8 while
> > parallel_seqscan_degree is 4. So, this may be a case different from Thom's.
> >>
> >
> > I think this is due to reason that memory for forming
> > tuple in master backend is retained for longer time which
> > is causing this statement to take much longer time than
> > required. I have fixed the other issue as well reported by
> > you in attached patch.
> >
>
> Thanks for fixing.
>
> > I think this patch is still not completely ready for general
> > purpose testing, however it could be helpful if we can run
> > some tests to see in what kind of scenario's it gives benefit
> > like in the test you are doing if rather than increasing
> > seq_page_cost, you should add an expensive WHERE condition
> > so that it should automatically select parallel plan. I think it is better
> > to change one of the new parameter's (parallel_setup_cost,
> > parallel_startup_cost and cpu_tuple_comm_cost) if you want
> > your statement to use parallel plan, like in your example if
> > you would have reduced cpu_tuple_comm_cost, it would have
> > selected parallel plan, that way we can get some feedback about
> > what should be the appropriate default values for the newly added
> > parameters. I am already planing to do some tests in that regard,
> > however if I get some feedback from other's that would be helpful.
> >
> >
>
> Perhaps you are aware or you've postponed working on it, but I see that
> a plan executing in a worker does not know about instrumentation.
> Note the "Rows Removed by Filter". I guess the difference may be
> because, all the rows filtered by workers were not accounted for. I'm
> not quite sure, but since exec_worker_stmt goes the Portal way,
> QueryDesc.instrument_options remains unset and hence no instrumentation
> opportunities in a worker backend. One option may be to pass
> instrument_options down to worker_stmt?
>
> By the way, 17s and 6s compare really well in favor of parallel seqscan
> above, :)
>
On 22-01-2015 PM 02:30, Amit Kapila wrote: >> Perhaps you are aware or you've postponed working on it, but I see that >> a plan executing in a worker does not know about instrumentation. > > I have deferred it until other main parts are stabilised/reviewed. Once > that is done, we can take a call what is best we can do for instrumentation. > Thom has reported the same as well upthread. > Ah, I missed Thom's report. >> Note the "Rows Removed by Filter". I guess the difference may be >> because, all the rows filtered by workers were not accounted for. I'm >> not quite sure, but since exec_worker_stmt goes the Portal way, >> QueryDesc.instrument_options remains unset and hence no instrumentation >> opportunities in a worker backend. One option may be to pass >> instrument_options down to worker_stmt? >> > > I think there is more to it, master backend need to process that information > as well. > I see. Thanks, Amit
>
> On Mon, Jan 19, 2015 at 2:24 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> > Another thing is that I think prefetching is not supported on all platforms
> > (Windows) and for such systems as per above algorithm we need to
> > rely on block-by-block method.
>
> Well, I think we should try to set up a test to see if this is hurting
> us. First, do a sequential-scan of a related too big at least twice
> as large as RAM. Then, do a parallel sequential scan of the same
> relation with 2 workers. Repeat these in alternation several times.
> If the operating system is accomplishing meaningful readahead, and the
> parallel sequential scan is breaking it, then since the test is
> I/O-bound I would expect to see the parallel scan actually being
> slower than the normal way.
>
I have taken some performance data as per above discussion. Basically,
IBM POWER-7 16 cores, 64 hardware threads
RAM = 64GB
Block-By-Block | ||
No. of workers/Time (ms) | 0 | 2 |
Run-1 | 267798 | 295051 |
Run-2 | 276646 | 296665 |
Run-3 | 281364 | 314952 |
Run-4 | 290231 | 326243 |
Run-5 | 288890 | 295684 |
Fixed-Chunks | ||
No. of workers/Time (ms) | 0 | 2 |
286346 | 234037 | |
250051 | 215111 | |
255915 | 254934 | |
263754 | 242228 | |
251399 | 202581 |
Attachment
On Thu, Jan 22, 2015 at 5:57 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > 1. Scanning block-by-block has negative impact on performance and > I thin it will degrade more if we increase parallel count as that can lead > to more randomness. > > 2. Scanning in fixed chunks improves the performance. Increasing > parallel count to a very large number might impact the performance, > but I think we can have a lower bound below which we will not allow > multiple processes to scan the relation. I'm confused. Your actual test numbers seem to show that the performance with the block-by-block approach was slightly higher with parallelism than without, where as the performance with the chunk-by-chunk approach was lower with parallelism than without, but the text quoted above, summarizing those numbers, says the opposite. Also, I think testing with 2 workers is probably not enough. I think we should test with 8 or even 16. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Jan 22, 2015 at 5:57 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > 1. Scanning block-by-block has negative impact on performance and
> > I thin it will degrade more if we increase parallel count as that can lead
> > to more randomness.
> >
> > 2. Scanning in fixed chunks improves the performance. Increasing
> > parallel count to a very large number might impact the performance,
> > but I think we can have a lower bound below which we will not allow
> > multiple processes to scan the relation.
>
> I'm confused. Your actual test numbers seem to show that the
> performance with the block-by-block approach was slightly higher with
> parallelism than without, where as the performance with the
> chunk-by-chunk approach was lower with parallelism than without, but
> the text quoted above, summarizing those numbers, says the opposite.
>
> Also, I think testing with 2 workers is probably not enough. I think
> we should test with 8 or even 16.
>
Sure, will do this and post the numbers.
On Thu, Jan 22, 2015 at 9:02 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I'm confused. Your actual test numbers seem to show that the >> performance with the block-by-block approach was slightly higher with >> parallelism than without, where as the performance with the >> chunk-by-chunk approach was lower with parallelism than without, but >> the text quoted above, summarizing those numbers, says the opposite. > > Sorry for causing confusion, I should have been more explicit about > explaining the numbers. Let me try again, > Values in columns is time in milliseconds to complete the execution, > so higher means it took more time. If you see in block-by-block, the > time taken to complete the execution with 2 workers is more than > no workers which means parallelism has degraded the performance. *facepalm* Oh, yeah, right. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 01/22/2015 05:53 AM, Robert Haas wrote: > Also, I think testing with 2 workers is probably not enough. I think > we should test with 8 or even 16. FWIW, based on my experience there will also be demand to use parallel query using 4 workers, particularly on AWS. -- Josh Berkus PostgreSQL Experts Inc. http://pgexperts.com
>
> On Thu, Jan 22, 2015 at 5:57 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > 1. Scanning block-by-block has negative impact on performance and
> > I thin it will degrade more if we increase parallel count as that can lead
> > to more randomness.
> >
> > 2. Scanning in fixed chunks improves the performance. Increasing
> > parallel count to a very large number might impact the performance,
> > but I think we can have a lower bound below which we will not allow
> > multiple processes to scan the relation.
>
> I'm confused. Your actual test numbers seem to show that the
> performance with the block-by-block approach was slightly higher with
> parallelism than without, where as the performance with the
> chunk-by-chunk approach was lower with parallelism than without, but
> the text quoted above, summarizing those numbers, says the opposite.
>
> Also, I think testing with 2 workers is probably not enough. I think
> we should test with 8 or even 16.
>
Block-By-Block | |||||||
No. of workers/Time (ms) | 0 | 2 | 4 | 8 | 16 | 24 | 32 |
Run-1 | 257851 | 287353 | 350091 | 330193 | 284913 | 338001 | 295057 |
Run-2 | 263241 | 314083 | 342166 | 347337 | 378057 | 351916 | 348292 |
Run-3 | 315374 | 334208 | 389907 | 340327 | 328695 | 330048 | 330102 |
Run-4 | 301054 | 312790 | 314682 | 352835 | 323926 | 324042 | 302147 |
Run-5 | 304547 | 314171 | 349158 | 350191 | 350468 | 341219 | 281315 |
Fixed-Chunks | |||||||
No. of workers/Time (ms) | 0 | 2 | 4 | 8 | 16 | 24 | 32 |
Run-1 | 250536 | 266279 | 251263 | 234347 | 87930 | 50474 | 35474 |
Run-2 | 249587 | 230628 | 225648 | 193340 | 83036 | 35140 | 9100 |
Run-3 | 234963 | 220671 | 230002 | 256183 | 105382 | 62493 | 27903 |
Run-4 | 239111 | 245448 | 224057 | 189196 | 123780 | 63794 | 24746 |
Run-5 | 239937 | 222820 | 219025 | 220478 | 114007 | 77965 | 39766 |
On 1/23/15 5:42 AM, Amit Kapila wrote: > *Fixed-Chunks* > *No. of workers/Time (ms)*> *0* *2* *4* *8* *16* *24* *32* > Run-1 250536 266279 251263 234347 87930 50474 35474 > Run-2 249587 230628 225648 193340 83036 35140 9100 > Run-3 234963 220671 230002 256183 105382 62493 27903 > Run-4 239111 245448 224057 189196 123780 63794 24746 > Run-5 239937 222820 219025 220478 114007 77965 39766 > > > > The trend remains same although there is some variation. > In block-by-block approach, it performance dips (execution takes > more time) with more number of workers, though it stabilizes at > some higher value, still I feel it is random as it leads to random > scan. > In Fixed-chunk approach, the performance improves with more > number of workers especially at slightly higher worker count. Those fixed chunk numbers look pretty screwy. 2, 4 and 8 workers make no difference, then suddenly 16 cuts times by 1/2 to1/3? Then 32 cuts time by another 1/2 to 1/3? -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
On 01/23/2015 10:44 AM, Jim Nasby wrote: > number of workers especially at slightly higher worker count. > > Those fixed chunk numbers look pretty screwy. 2, 4 and 8 workers make no > difference, then suddenly 16 cuts times by 1/2 to 1/3? Then 32 cuts time > by another 1/2 to 1/3? cached? First couple of runs gets the relations into memory? JD -- Command Prompt, Inc. - http://www.commandprompt.com/ 503-667-4564 PostgreSQL Support, Training, Professional Services and Development High Availability, Oracle Conversion, @cmdpromptinc "If we send our children to Caesar for their education, we should not be surprised when they come back as Romans."
>
>
> On 01/23/2015 10:44 AM, Jim Nasby wrote:
>>
>> number of workers especially at slightly higher worker count.
>>
>> Those fixed chunk numbers look pretty screwy. 2, 4 and 8 workers make no
>> difference, then suddenly 16 cuts times by 1/2 to 1/3? Then 32 cuts time
>> by another 1/2 to 1/3?
>
>
> cached? First couple of runs gets the relations into memory?
>
On 1/23/15 10:16 PM, Amit Kapila wrote: > Further, if we want to just get the benefit of parallel I/O, then > I think we can get that by parallelising partition scan where different > table partitions reside on different disk partitions, however that is > a matter of separate patch. I don't think we even have to go that far. My experience with Postgres is that it is *very* sensitive to IO latency (not bandwidth). I believe this is the case becausecomplex queries tend to interleave CPU intensive code in-between IO requests. So we see this pattern: Wait 5ms on IO Compute for a few ms Wait 5ms on IO Compute for a few ms ... We blindly assume that the kernel will magically do read-ahead for us, but I've never seen that work so great. It certainlyfalls apart on something like an index scan. If we could instead do this: Wait for first IO, issue second IO request Compute Already have second IO request, issue third ... We'd be a lot less sensitive to IO latency. I wonder what kind of gains we would see if every SeqScan in a query spawned a worker just to read tuples and shove themin a queue (or shove a pointer to a buffer in the queue). Similarly, have IndexScans have one worker reading the indexand another worker taking index tuples and reading heap tuples... -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
Jim Nasby <Jim.Nasby@BlueTreble.com> writes: > On 1/23/15 10:16 PM, Amit Kapila wrote: >> Further, if we want to just get the benefit of parallel I/O, then >> I think we can get that by parallelising partition scan where different >> table partitions reside on different disk partitions, however that is >> a matter of separate patch. > I don't think we even have to go that far. > My experience with Postgres is that it is *very* sensitive to IO latency (not bandwidth). I believe this is the case becausecomplex queries tend to interleave CPU intensive code in-between IO requests. So we see this pattern: > Wait 5ms on IO > Compute for a few ms > Wait 5ms on IO > Compute for a few ms > ... > We blindly assume that the kernel will magically do read-ahead for us, but I've never seen that work so great. It certainlyfalls apart on something like an index scan. > If we could instead do this: > Wait for first IO, issue second IO request > Compute > Already have second IO request, issue third > ... > We'd be a lot less sensitive to IO latency. It would take about five minutes of coding to prove or disprove this: stick a PrefetchBuffer call into heapgetpage() to launch a request for the next page as soon as we've read the current one, and then see if that makes any obvious performance difference. I'm not convinced that it will, but if it did then we could think about how to make it work for real. regards, tom lane
>
> On 1/23/15 10:16 PM, Amit Kapila wrote:
>>
>> Further, if we want to just get the benefit of parallel I/O, then
>> I think we can get that by parallelising partition scan where different
>> table partitions reside on different disk partitions, however that is
>> a matter of separate patch.
>
>
> I don't think we even have to go that far.
>
>
> We'd be a lot less sensitive to IO latency.
>
> I wonder what kind of gains we would see if every SeqScan in a query spawned a worker just to read tuples and shove them in a queue (or shove a pointer to a buffer in the queue).
>
Here IIUC, you want to say that just get the read done by one parallel
Hi PG devs! Tom Lane <tgl@sss.pgh.pa.us> writes: >> Wait for first IO, issue second IO request >> Compute >> Already have second IO request, issue third >> ... > >> We'd be a lot less sensitive to IO latency. > > It would take about five minutes of coding to prove or disprove this: > stick a PrefetchBuffer call into heapgetpage() to launch a request for the > next page as soon as we've read the current one, and then see if that > makes any obvious performance difference. I'm not convinced that it will, > but if it did then we could think about how to make it work for real. Sorry for dropping in so late... I have done all this two years ago. For TPC-H Q8, Q9, Q17, Q20, and Q21 I see a speedup of ~100% when using IndexScan prefetching + Nested-Loops Look-Ahead (the outer loop!). (On SSD with 32 Pages Prefetch/Look-Ahead + Cold Page Cache / Small RAM) Regards, Daniel -- MSc. Daniel Bausch Research Assistant (Computer Science) Technische Universität Darmstadt http://www.dvs.tu-darmstadt.de/staff/dbausch
On Tue, Jan 27, 2015 at 08:02:37AM +0100, Daniel Bausch wrote: > Hi PG devs! > > Tom Lane <tgl@sss.pgh.pa.us> writes: > > >> Wait for first IO, issue second IO request > >> Compute > >> Already have second IO request, issue third > >> ... > > > >> We'd be a lot less sensitive to IO latency. > > > > It would take about five minutes of coding to prove or disprove this: > > stick a PrefetchBuffer call into heapgetpage() to launch a request for the > > next page as soon as we've read the current one, and then see if that > > makes any obvious performance difference. I'm not convinced that it will, > > but if it did then we could think about how to make it work for real. > > Sorry for dropping in so late... > > I have done all this two years ago. For TPC-H Q8, Q9, Q17, Q20, and Q21 > I see a speedup of ~100% when using IndexScan prefetching + Nested-Loops > Look-Ahead (the outer loop!). > (On SSD with 32 Pages Prefetch/Look-Ahead + Cold Page Cache / Small RAM) Would you be so kind as to pass along any patches (ideally applicable to git master), tests, and specific measurements you made? Cheers, David. -- David Fetter <david@fetter.org> http://fetter.org/ Phone: +1 415 235 3778 AIM: dfetter666 Yahoo!: dfetter Skype: davidfetter XMPP: david.fetter@gmail.com Remember to vote! Consider donating to Postgres: http://www.postgresql.org/about/donate
On Thu, Jan 22, 2015 at 5:57 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Script used to test is attached (parallel_count.sh) Why does this use EXPLAIN ANALYZE instead of \timing ? > IBM POWER-7 16 cores, 64 hardware threads > RAM = 64GB > > Table Size - 120GB > > Used below statements to create table - > create table tbl_perf(c1 int, c2 char(1000)); > insert into tbl_perf values(generate_series(1,10000000),'aaaaa'); > insert into tbl_perf values(generate_series(10000001,30000000),'aaaaa'); > insert into tbl_perf values(generate_series(30000001,110000000),'aaaaa'); I generated this table using this same method and experimented with copying the whole file to the bit bucket using dd. I did this on hydra, which I think is the same machine you used. time for i in `seq 0 119`; do if [ $i -eq 0 ]; then f=16388; else f=16388.$i; fi; dd if=$f of=/dev/null bs=8k; done There is a considerable amount of variation in the amount of time this takes to run based on how much of the relation is cached. Clearly, there's no way for the system to cache it all, but it can cache a significant portion, and that affects the results to no small degree. dd on hydra prints information on the data transfer rate; on uncached 1GB segments, it runs at right around 400 MB/s, but that can soar to upwards of 3GB/s when the relation is fully cached. I tried flushing the OS cache via echo 1 > /proc/sys/vm/drop_caches, and found that immediately after doing that, the above command took 5m21s to run - i.e. ~321000 ms. Most of your test times are faster than that, which means they reflect some degree of caching. When I immediately reran the command a second time, it finished in 4m18s the second time, or ~258000 ms. The rate was the same as the first test - about 400 MB/s - for most of the files, but 27 of the last 28 files went much faster, between 1.3 GB/s and 3.7 GB/s. This tells us that the OS cache on this machine has anti-spoliation logic in it, probably not dissimilar to what we have in PG. If the data were cycled through the system cache in strict LRU fashion, any data that was leftover from the first run would have been flushed out by the early part of the second run, so that all the results from the second set of runs would have hit the disk. But in fact, that's not what happened: the last pages from the first run remained cached even after reading an amount of new data that exceeds the size of RAM on that machine. What I think this demonstrates is that we're going to have to be very careful to control for caching effects, or we may find that we get misleading results. To make this simpler, I've installed a setuid binary /usr/bin/drop_caches that you (or anyone who has an account on that machine) can use you drop the caches; run 'drop_caches 1'. > Block-By-Block > > No. of workers/Time (ms) 0 2 > Run-1 267798 295051 > Run-2 276646 296665 > Run-3 281364 314952 > Run-4 290231 326243 > Run-5 288890 295684 The next thing I did was run test with the block-by-block method after having dropped the caches. I did this with 0 workers and with 8 workers. I dropped the caches and restarted postgres before each test, but then ran each test a second time to see the effect of caching by both the OS and by PostgreSQL. I got these results: With 0 workers, first run took 883465.352 ms, and second run took 295050.106 ms. With 8 workers, first run took 340302.250 ms, and second run took 307767.758 ms. This is a confusing result, because you expect parallelism to help more when the relation is partly cached, and make little or no difference when it isn't cached. But that's not what happened. I've also got a draft of a prefetching implementation here that I'd like to test out, but I've just discovered that it's buggy, so I'm going to send these results for now and work on fixing that. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Robert, all, * Robert Haas (robertmhaas@gmail.com) wrote: > There is a considerable amount of variation in the amount of time this > takes to run based on how much of the relation is cached. Clearly, > there's no way for the system to cache it all, but it can cache a > significant portion, and that affects the results to no small degree. > dd on hydra prints information on the data transfer rate; on uncached > 1GB segments, it runs at right around 400 MB/s, but that can soar to > upwards of 3GB/s when the relation is fully cached. I tried flushing > the OS cache via echo 1 > /proc/sys/vm/drop_caches, and found that > immediately after doing that, the above command took 5m21s to run - > i.e. ~321000 ms. Most of your test times are faster than that, which > means they reflect some degree of caching. When I immediately reran > the command a second time, it finished in 4m18s the second time, or > ~258000 ms. The rate was the same as the first test - about 400 MB/s > - for most of the files, but 27 of the last 28 files went much faster, > between 1.3 GB/s and 3.7 GB/s. [...] > With 0 workers, first run took 883465.352 ms, and second run took 295050.106 ms. > With 8 workers, first run took 340302.250 ms, and second run took 307767.758 ms. > > This is a confusing result, because you expect parallelism to help > more when the relation is partly cached, and make little or no > difference when it isn't cached. But that's not what happened. These numbers seem to indicate that the oddball is the single-threaded uncached run. If I followed correctly, the uncached 'dd' took 321s, which is relatively close to the uncached-lots-of-workers and the two cached runs. What in the world is the uncached single-thread case doing that it takes an extra 543s, or over twice as long? It's clearly not disk i/o which is causing the slowdown, based on your dd tests. One possibility might be round-trip latency. The multi-threaded case is able to keep the CPUs and the i/o system going, and the cached results don't have as much latency since things are cached, but the single-threaded uncached case going i/o -> cpu -> i/o -> cpu, ends up with a lot of wait time as it switches between being on CPU and waiting on the i/o. Just some thoughts. Thanks, Stephen
On Fri, Jan 23, 2015 at 6:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Fixed-Chunks > > No. of workers/Time (ms) 0 2 4 8 16 24 32 > Run-1 250536 266279 251263 234347 87930 50474 35474 > Run-2 249587 230628 225648 193340 83036 35140 9100 > Run-3 234963 220671 230002 256183 105382 62493 27903 > Run-4 239111 245448 224057 189196 123780 63794 24746 > Run-5 239937 222820 219025 220478 114007 77965 39766 I cannot reproduce these results. I applied your fixed-chunk size patch and ran SELECT parallel_count('tbl_perf', 32) a few times. The first thing I notice is that, as I predicted, there's an issue with different workers finishing at different times. For example, from my first run: 2015-01-27 22:13:09 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34700) exited with exit code 0 2015-01-27 22:13:09 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34698) exited with exit code 0 2015-01-27 22:13:09 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34701) exited with exit code 0 2015-01-27 22:13:10 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34699) exited with exit code 0 2015-01-27 22:15:00 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34683) exited with exit code 0 2015-01-27 22:15:29 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34673) exited with exit code 0 2015-01-27 22:15:58 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34679) exited with exit code 0 2015-01-27 22:16:38 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34689) exited with exit code 0 2015-01-27 22:16:39 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34671) exited with exit code 0 2015-01-27 22:16:47 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34677) exited with exit code 0 2015-01-27 22:16:47 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34672) exited with exit code 0 2015-01-27 22:16:48 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34680) exited with exit code 0 2015-01-27 22:16:50 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34686) exited with exit code 0 2015-01-27 22:16:51 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34670) exited with exit code 0 2015-01-27 22:16:51 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34690) exited with exit code 0 2015-01-27 22:16:51 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34674) exited with exit code 0 2015-01-27 22:16:52 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34684) exited with exit code 0 2015-01-27 22:16:53 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34675) exited with exit code 0 2015-01-27 22:16:53 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34682) exited with exit code 0 2015-01-27 22:16:53 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34691) exited with exit code 0 2015-01-27 22:16:54 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34676) exited with exit code 0 2015-01-27 22:16:54 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34685) exited with exit code 0 2015-01-27 22:16:55 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34692) exited with exit code 0 2015-01-27 22:16:56 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34687) exited with exit code 0 2015-01-27 22:16:56 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34678) exited with exit code 0 2015-01-27 22:16:57 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34681) exited with exit code 0 2015-01-27 22:16:57 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34688) exited with exit code 0 2015-01-27 22:16:59 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34694) exited with exit code 0 2015-01-27 22:16:59 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34693) exited with exit code 0 2015-01-27 22:17:02 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34695) exited with exit code 0 2015-01-27 22:17:02 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34697) exited with exit code 0 2015-01-27 22:17:02 UTC [34660] LOG: worker process: parallel worker for PID 34668 (PID 34696) exited with exit code 0 That run started at 22:13:01. Within 4 seconds, 4 workers exited. So clearly we are not getting the promised 32-way parallelism for the whole test. Granted, in this instance, *most* of the workers run until the end, but I think we'll find that there are uncomfortably-frequent cases where we get significantly less parallelism than we planned on because the work isn't divided evenly. But leaving that aside, I've run this test 6 times in a row now, with a warm cache, and the best time I have is 237310.042 ms and the worst time I have is 242936.315 ms. So there's very little variation, and it's reasonably close to the results I got with dd, suggesting that the system is fairly well I/O bound. At a sequential read speed of 400 MB/s, 240 s = 96 GB of data. Assuming it takes no time at all to process the cached data (which seems to be not far from wrong judging by how quickly the first few workers exit), that means we're getting 24 GB of data from cache on a 64 GB machine. That seems a little low, but if the kernel is refusing to cache the whole relation to avoid cache-trashing, it could be right. Now, when you did what I understand to be the same test on the same machine, you got times ranging from 9.1 seconds to 35.4 seconds. Clearly, there is some difference between our test setups. Moreover, I'm kind of suspicious about whether your results are actually physically possible. Even in the best case where you somehow had the maximum possible amount of data - 64 GB on a 64 GB machine - cached, leaving no space for cache duplication between PG and the OS and no space for the operating system or postgres itself - the table is 120 GB, so you've got to read *at least* 56 GB from disk. Reading 56 GB from disk in 9 seconds represents an I/O rate of >6 GB/s. I grant that there could be some speedup from issuing I/O requests in parallel instead of serially, but that is a 15x speedup over dd, so I am a little suspicious that there is some problem with the test setup, especially because I cannot reproduce the results. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 1/26/15 11:11 PM, Amit Kapila wrote: > On Tue, Jan 27, 2015 at 3:18 AM, Jim Nasby <Jim.Nasby@bluetreble.com <mailto:Jim.Nasby@bluetreble.com>> wrote: > > > > On 1/23/15 10:16 PM, Amit Kapila wrote: > >> > >> Further, if we want to just get the benefit of parallel I/O, then > >> I think we can get that by parallelising partition scan where different > >> table partitions reside on different disk partitions, however that is > >> a matter of separate patch. > > > > > > I don't think we even have to go that far. > > > > > > We'd be a lot less sensitive to IO latency. > > > > I wonder what kind of gains we would see if every SeqScan in a query spawned a worker just to read tuples and shovethem in a queue (or shove a pointer to a buffer in the queue). > > > > Here IIUC, you want to say that just get the read done by one parallel > worker and then all expression calculation (evaluation of qualification > and target list) in the main backend, it seems to me that by doing it > that way, the benefit of parallelisation will be lost due to tuple > communication overhead (may be the overhead is less if we just > pass a pointer to buffer but that will have another kind of problems > like holding buffer pins for a longer period of time). > > I could see the advantage of testing on lines as suggested by Tom Lane, > but that seems to be not directly related to what we want to achieve by > this patch (parallel seq scan) or if you think otherwise then let me know? There's some low-hanging fruit when it comes to improving our IO performance (or more specifically, decreasing our sensitivityto IO latency). Perhaps the way to do that is with the parallel infrastructure, perhaps not. But I think it'spremature to look at parallelism for increasing IO performance, or worrying about things like how many IO threads weshould have before we at least look at simpler things we could do. We shouldn't assume there's nothing to be gained shortof a full parallelization implementation. That's not to say there's nothing else we could use parallelism for. Sort, merge and hash operations come to mind. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
On 1/27/15 3:46 PM, Stephen Frost wrote: >> With 0 workers, first run took 883465.352 ms, and second run took 295050.106 ms. >> >With 8 workers, first run took 340302.250 ms, and second run took 307767.758 ms. >> > >> >This is a confusing result, because you expect parallelism to help >> >more when the relation is partly cached, and make little or no >> >difference when it isn't cached. But that's not what happened. > These numbers seem to indicate that the oddball is the single-threaded > uncached run. If I followed correctly, the uncached 'dd' took 321s, > which is relatively close to the uncached-lots-of-workers and the two > cached runs. What in the world is the uncached single-thread case doing > that it takes an extra 543s, or over twice as long? It's clearly not > disk i/o which is causing the slowdown, based on your dd tests. > > One possibility might be round-trip latency. The multi-threaded case is > able to keep the CPUs and the i/o system going, and the cached results > don't have as much latency since things are cached, but the > single-threaded uncached case going i/o -> cpu -> i/o -> cpu, ends up > with a lot of wait time as it switches between being on CPU and waiting > on the i/o. This exactly mirrors what I've seen on production systems. On a single SeqScan I can't get anywhere close to the IO performanceI could get with dd. Once I got up to 4-8 SeqScans of different tables running together, I saw iostat numbersthat were similar to what a single dd bs=8k would do. I've tested this with iSCSI SAN volumes on both 1Gbit and 10Gbitethernet. This is why I think that when it comes to IO performance, before we start worrying about real parallelization we should investigateways to do some kind of async IO. I only have my SSD laptop and a really old server to test on, but I'll try Tom's suggestion of adding a PrefetchBuffer callinto heapgetpage() unless someone beats me to it. I should be able to do it tomorrow. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
On Tue, Jan 27, 2015 at 4:46 PM, Stephen Frost <sfrost@snowman.net> wrote: >> With 0 workers, first run took 883465.352 ms, and second run took 295050.106 ms. >> With 8 workers, first run took 340302.250 ms, and second run took 307767.758 ms. >> >> This is a confusing result, because you expect parallelism to help >> more when the relation is partly cached, and make little or no >> difference when it isn't cached. But that's not what happened. > > These numbers seem to indicate that the oddball is the single-threaded > uncached run. If I followed correctly, the uncached 'dd' took 321s, > which is relatively close to the uncached-lots-of-workers and the two > cached runs. What in the world is the uncached single-thread case doing > that it takes an extra 543s, or over twice as long? It's clearly not > disk i/o which is causing the slowdown, based on your dd tests. Yeah, I'm wondering if the disk just froze up on that run for a long while, which has been known to occasionally happen on this machine, because I can't reproduce that crappy number. I did the 0-worker test a few more times, with the block-by-block method, dropping the caches and restarting PostgreSQL each time, and got: 322222.968 ms 322873.325 ms 322967.722 ms 321759.273 ms After that last run, I ran it a few more times without restarting PostgreSQL or dropping the caches, and got: 257629.348 ms 289668.976 ms 290342.970 ms 258035.226 ms 284237.729 ms Then I redid the 8-client test. Cold cache, I got 337312.554 ms. On the rerun, 323423.813 ms. Third run, 324940.785. There is more variability than I would like here. Clearly, it goes a bit faster when the cache is warm, but that's about all I can say with any confidence. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Tue, Jan 27, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote: > Now, when you did what I understand to be the same test on the same > machine, you got times ranging from 9.1 seconds to 35.4 seconds. > Clearly, there is some difference between our test setups. Moreover, > I'm kind of suspicious about whether your results are actually > physically possible. Even in the best case where you somehow had the > maximum possible amount of data - 64 GB on a 64 GB machine - cached, > leaving no space for cache duplication between PG and the OS and no > space for the operating system or postgres itself - the table is 120 > GB, so you've got to read *at least* 56 GB from disk. Reading 56 GB > from disk in 9 seconds represents an I/O rate of >6 GB/s. I grant that > there could be some speedup from issuing I/O requests in parallel > instead of serially, but that is a 15x speedup over dd, so I am a > little suspicious that there is some problem with the test setup, > especially because I cannot reproduce the results. So I thought about this a little more, and I realized after some poking around that hydra's disk subsystem is actually six disks configured in a software RAID5[1]. So one advantage of the chunk-by-chunk approach you are proposing is that you might be able to get all of the disks chugging away at once, because the data is presumably striped across all of them. Reading one block at a time, you'll never have more than 1 or 2 disks going, but if you do sequential reads from a bunch of different places in the relation, you might manage to get all 6. So that's something to think about. One could imagine an algorithm like this: as long as there are more 1GB segments remaining than there are workers, each worker tries to chug through a separate 1GB segment. When there are not enough 1GB segments remaining for that to work, then they start ganging up on the same segments. That way, you get the benefit of spreading out the I/O across multiple files (and thus hopefully multiple members of the RAID group) when the data is coming from disk, but you can still keep everyone busy until the end, which will be important when the data is all in-memory and you're just limited by CPU bandwidth. All that aside, I still can't account for the numbers you are seeing. When I run with your patch and what I think is your test case, I get different (slower) numbers. And even if we've got 6 drives cranking along at 400MB/s each, that's still only 2.4 GB/s, not >6 GB/s. So I'm still perplexed. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company [1] Not my idea.
On 01/28/2015 04:16 AM, Robert Haas wrote: > On Tue, Jan 27, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> Now, when you did what I understand to be the same test on the same >> machine, you got times ranging from 9.1 seconds to 35.4 seconds. >> Clearly, there is some difference between our test setups. Moreover, >> I'm kind of suspicious about whether your results are actually >> physically possible. Even in the best case where you somehow had the >> maximum possible amount of data - 64 GB on a 64 GB machine - cached, >> leaving no space for cache duplication between PG and the OS and no >> space for the operating system or postgres itself - the table is 120 >> GB, so you've got to read *at least* 56 GB from disk. Reading 56 GB >> from disk in 9 seconds represents an I/O rate of >6 GB/s. I grant that >> there could be some speedup from issuing I/O requests in parallel >> instead of serially, but that is a 15x speedup over dd, so I am a >> little suspicious that there is some problem with the test setup, >> especially because I cannot reproduce the results. > > So I thought about this a little more, and I realized after some > poking around that hydra's disk subsystem is actually six disks > configured in a software RAID5[1]. So one advantage of the > chunk-by-chunk approach you are proposing is that you might be able to > get all of the disks chugging away at once, because the data is > presumably striped across all of them. Reading one block at a time, > you'll never have more than 1 or 2 disks going, but if you do > sequential reads from a bunch of different places in the relation, you > might manage to get all 6. So that's something to think about. > > One could imagine an algorithm like this: as long as there are more > 1GB segments remaining than there are workers, each worker tries to > chug through a separate 1GB segment. When there are not enough 1GB > segments remaining for that to work, then they start ganging up on the > same segments. That way, you get the benefit of spreading out the I/O > across multiple files (and thus hopefully multiple members of the RAID > group) when the data is coming from disk, but you can still keep > everyone busy until the end, which will be important when the data is > all in-memory and you're just limited by CPU bandwidth. OTOH, spreading the I/O across multiple files is not a good thing, if you don't have a RAID setup like that. With a single spindle, you'll just induce more seeks. Perhaps the OS is smart enough to read in large-enough chunks that the occasional seek doesn't hurt much. But then again, why isn't the OS smart enough to read in large-enough chunks to take advantage of the RAID even when you read just a single file? - Heikki
>
> On 01/28/2015 04:16 AM, Robert Haas wrote:
>>
>> On Tue, Jan 27, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:
>>>
>>> Now, when you did what I understand to be the same test on the same
>>> machine, you got times ranging from 9.1 seconds to 35.4 seconds.
>>> Clearly, there is some difference between our test setups. Moreover,
>>> I'm kind of suspicious about whether your results are actually
>>> physically possible. Even in the best case where you somehow had the
>>> maximum possible amount of data - 64 GB on a 64 GB machine - cached,
>>> leaving no space for cache duplication between PG and the OS and no
>>> space for the operating system or postgres itself - the table is 120
>>> GB, so you've got to read *at least* 56 GB from disk. Reading 56 GB
>>> from disk in 9 seconds represents an I/O rate of >6 GB/s. I grant that
>>> there could be some speedup from issuing I/O requests in parallel
>>> instead of serially, but that is a 15x speedup over dd, so I am a
>>> little suspicious that there is some problem with the test setup,
>>> especially because I cannot reproduce the results.
>>
>>
>> So I thought about this a little more, and I realized after some
>> poking around that hydra's disk subsystem is actually six disks
>> configured in a software RAID5[1]. So one advantage of the
>> chunk-by-chunk approach you are proposing is that you might be able to
>> get all of the disks chugging away at once, because the data is
>> presumably striped across all of them. Reading one block at a time,
>> you'll never have more than 1 or 2 disks going, but if you do
>> sequential reads from a bunch of different places in the relation, you
>> might manage to get all 6. So that's something to think about.
>>
>> One could imagine an algorithm like this: as long as there are more
>> 1GB segments remaining than there are workers, each worker tries to
>> chug through a separate 1GB segment. When there are not enough 1GB
>> segments remaining for that to work, then they start ganging up on the
>> same segments. That way, you get the benefit of spreading out the I/O
>> across multiple files (and thus hopefully multiple members of the RAID
>> group) when the data is coming from disk, but you can still keep
>> everyone busy until the end, which will be important when the data is
>> all in-memory and you're just limited by CPU bandwidth.
>
>
> OTOH, spreading the I/O across multiple files is not a good thing, if you don't have a RAID setup like that. With a single spindle, you'll just induce more seeks.
>
Yeah, if such a thing happens then there is less chance that user
On Wed, Jan 28, 2015 at 2:08 AM, Heikki Linnakangas <hlinnakangas@vmware.com> wrote: > OTOH, spreading the I/O across multiple files is not a good thing, if you > don't have a RAID setup like that. With a single spindle, you'll just induce > more seeks. > > Perhaps the OS is smart enough to read in large-enough chunks that the > occasional seek doesn't hurt much. But then again, why isn't the OS smart > enough to read in large-enough chunks to take advantage of the RAID even > when you read just a single file? Suppose we have N spindles and N worker processes and it just so happens that the amount of computation is such that a each spindle can keep one CPU busy. Let's suppose the chunk size is 4MB. If you read from the relation at N staggered offsets, you might be lucky enough that each one of them keeps a spindle busy, and you might be lucky enough to have that stay true as the scans advance. You don't need any particularly large amount of read-ahead; you just need to stay at least one block ahead of the CPU. But if you read the relation in one pass from beginning to end, you need at least N*4MB of read-ahead to have data in cache for all N spindles, and the read-ahead will certainly fail you at the end of every 1GB segment. The problem here, as I see it, is that we're flying blind. If there's just one spindle, I think it's got to be right to read the relation sequentially. But if there are multiple spindles, it might not be, but it seems hard to predict what we should do. We don't know what the RAID chunk size is or how many spindles there are, so any guess as to how to chunk up the relation and divide up the work between workers is just a shot in the dark. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
<div dir="ltr"><div class="gmail_extra"><div class="gmail_quote">On 28 January 2015 at 14:03, Robert Haas <span dir="ltr"><<ahref="mailto:robertmhaas@gmail.com" target="_blank">robertmhaas@gmail.com</a>></span> wrote:<br /><blockquoteclass="gmail_quote" style="margin:0 0 0 .8ex;border-left:1px #ccc solid;padding-left:1ex">The problem here,as I see it, is that we're flying blind. If there's<br /> just one spindle, I think it's got to be right to read therelation<br /> sequentially. But if there are multiple spindles, it might not be,<br /> but it seems hard to predictwhat we should do. We don't know what<br /> the RAID chunk size is or how many spindles there are, so any guess as<br/> to how to chunk up the relation and divide up the work between workers<br /> is just a shot in the dark.</blockquote></div><br/></div><div class="gmail_extra">Can't the planner take effective_io_concurrency into account?<brclear="all" /></div><div class="gmail_extra"><br /><div class="gmail_signature">Thom</div></div></div>
>
>
> All that aside, I still can't account for the numbers you are seeing.
> When I run with your patch and what I think is your test case, I get
> different (slower) numbers. And even if we've got 6 drives cranking
> along at 400MB/s each, that's still only 2.4 GB/s, not >6 GB/s. So
> I'm still perplexed.
>
I have tried the tests again and found that I have forgotten to increase
On Wed, Jan 28, 2015 at 9:12 AM, Thom Brown <thom@linux.com> wrote: > On 28 January 2015 at 14:03, Robert Haas <robertmhaas@gmail.com> wrote: >> The problem here, as I see it, is that we're flying blind. If there's >> just one spindle, I think it's got to be right to read the relation >> sequentially. But if there are multiple spindles, it might not be, >> but it seems hard to predict what we should do. We don't know what >> the RAID chunk size is or how many spindles there are, so any guess as >> to how to chunk up the relation and divide up the work between workers >> is just a shot in the dark. > > Can't the planner take effective_io_concurrency into account? Maybe. It's answering a somewhat the right question -- to tell us how many parallel I/O channels we think we've got. But I'm not quite sure what the to do with that information in this case. I mean, if we've got effective_io_concurrency = 6, does that mean it's right to start scans in 6 arbitrary places in the relation and hope that keeps all the drives busy? That seems like throwing darts at the wall. We have no idea which parts are on which underlying devices. Or maybe it mean we should prefetch 24MB, on the assumption that the RAID stripe is 4MB? That's definitely blind guesswork. Considering the email Amit just sent, it looks like on this machine, regardless of what algorithm we used, the scan took between 3 minutes and 5.5 minutes, and most of them took between 4 minutes and 5.5 minutes. The results aren't very predictable, more workers don't necessarily help, and it's not really clear that any algorithm we've tried is clearly better than any other. I experimented with prefetching a bit yesterday, too, and it was pretty much the same. Some settings made it slightly faster. Others made it slower. Whee! -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Robert Haas <robertmhaas@gmail.com> writes: > The problem here, as I see it, is that we're flying blind. If there's > just one spindle, I think it's got to be right to read the relation > sequentially. But if there are multiple spindles, it might not be, > but it seems hard to predict what we should do. We don't know what > the RAID chunk size is or how many spindles there are, so any guess as > to how to chunk up the relation and divide up the work between workers > is just a shot in the dark. I thought the proposal to chunk on the basis of "each worker processes one 1GB-sized segment" should work all right. The kernel should see that as sequential reads of different files, issued by different processes; and if it can't figure out how to process that efficiently then it's a very sad excuse for a kernel. You are right that trying to do any detailed I/O scheduling by ourselves is a doomed exercise. For better or worse, we have kept ourselves at sufficient remove from the hardware that we can't possibly do that successfully. regards, tom lane
On Wed, Jan 28, 2015 at 10:40 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: > Robert Haas <robertmhaas@gmail.com> writes: >> The problem here, as I see it, is that we're flying blind. If there's >> just one spindle, I think it's got to be right to read the relation >> sequentially. But if there are multiple spindles, it might not be, >> but it seems hard to predict what we should do. We don't know what >> the RAID chunk size is or how many spindles there are, so any guess as >> to how to chunk up the relation and divide up the work between workers >> is just a shot in the dark. > > I thought the proposal to chunk on the basis of "each worker processes > one 1GB-sized segment" should work all right. The kernel should see that > as sequential reads of different files, issued by different processes; > and if it can't figure out how to process that efficiently then it's a > very sad excuse for a kernel. I agree. But there's only value in doing something like that if we have evidence that it improves anything. Such evidence is presently a bit thin on the ground. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Robert Haas <robertmhaas@gmail.com> writes: > On Wed, Jan 28, 2015 at 10:40 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: >> I thought the proposal to chunk on the basis of "each worker processes >> one 1GB-sized segment" should work all right. The kernel should see that >> as sequential reads of different files, issued by different processes; >> and if it can't figure out how to process that efficiently then it's a >> very sad excuse for a kernel. > I agree. But there's only value in doing something like that if we > have evidence that it improves anything. Such evidence is presently a > bit thin on the ground. Well, of course none of this should get committed without convincing evidence that it's a win. But I think that chunking on relation segment boundaries is a plausible way of dodging the problem that we can't do explicitly hardware-aware scheduling. regards, tom lane
* Robert Haas (robertmhaas@gmail.com) wrote: > On Wed, Jan 28, 2015 at 10:40 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: > > I thought the proposal to chunk on the basis of "each worker processes > > one 1GB-sized segment" should work all right. The kernel should see that > > as sequential reads of different files, issued by different processes; > > and if it can't figure out how to process that efficiently then it's a > > very sad excuse for a kernel. Agreed. > I agree. But there's only value in doing something like that if we > have evidence that it improves anything. Such evidence is presently a > bit thin on the ground. You need an i/o subsystem that's fast enough to keep a single CPU busy, otherwise (as you mentioned elsewhere), you're just going to be i/o bound and having more processes isn't going to help (and could hurt). Such i/o systems do exist, but a single RAID5 group over spinning rust with a simple filter isn't going to cut it with a modern CPU- we're just too darn efficient to end up i/o bound in that case. A more complex filter might be able to change it over to being more CPU bound than i/o bound and produce the performance improvments you're looking for. The caveat to this is if you have multiple i/o *channels* (which it looks like you don't in this case) where you can parallelize across those channels by having multiple processes involved. We only support multiple i/o channels today with tablespaces and we can't span tables across tablespaces. That's a problem when working with large data sets, but I'm hopeful that this work will eventually lead to a parallelized Append node that operates against a partitioned/inheirited table to work across multiple tablespaces. Thanks, Stephen
* Stephen Frost (sfrost@snowman.net) wrote: > Such i/o systems do exist, but a single RAID5 group over spinning rust > with a simple filter isn't going to cut it with a modern CPU- we're just > too darn efficient to end up i/o bound in that case. err, to *not* end up i/o bound. Thanks, Stephen
On 1/28/15 9:56 AM, Stephen Frost wrote: > * Robert Haas (robertmhaas@gmail.com) wrote: >> On Wed, Jan 28, 2015 at 10:40 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: >>> I thought the proposal to chunk on the basis of "each worker processes >>> one 1GB-sized segment" should work all right. The kernel should see that >>> as sequential reads of different files, issued by different processes; >>> and if it can't figure out how to process that efficiently then it's a >>> very sad excuse for a kernel. > > Agreed. > >> I agree. But there's only value in doing something like that if we >> have evidence that it improves anything. Such evidence is presently a >> bit thin on the ground. > > You need an i/o subsystem that's fast enough to keep a single CPU busy, > otherwise (as you mentioned elsewhere), you're just going to be i/o > bound and having more processes isn't going to help (and could hurt). > > Such i/o systems do exist, but a single RAID5 group over spinning rust > with a simple filter isn't going to cut it with a modern CPU- we're just > too darn efficient to end up i/o bound in that case. A more complex > filter might be able to change it over to being more CPU bound than i/o > bound and produce the performance improvments you're looking for. Except we're nowhere near being IO efficient. The vast difference between Postgres IO rates and dd shows this. I suspectthat's because we're not giving the OS a list of IO to perform while we're doing our thing, but that's just a guess. > The caveat to this is if you have multiple i/o *channels* (which it > looks like you don't in this case) where you can parallelize across > those channels by having multiple processes involved. Keep in mind that multiple processes is in no way a requirement for that. Async IO would do that, or even just requestingstuff from the OS before we need it. > We only support > multiple i/o channels today with tablespaces and we can't span tables > across tablespaces. That's a problem when working with large data sets, > but I'm hopeful that this work will eventually lead to a parallelized > Append node that operates against a partitioned/inheirited table to work > across multiple tablespaces. Until we can get a single seqscan close to dd performance, I fear worrying about tablespaces and IO channels is entirelypremature. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
Jim, * Jim Nasby (Jim.Nasby@BlueTreble.com) wrote: > On 1/28/15 9:56 AM, Stephen Frost wrote: > >Such i/o systems do exist, but a single RAID5 group over spinning rust > >with a simple filter isn't going to cut it with a modern CPU- we're just > >too darn efficient to end up i/o bound in that case. A more complex > >filter might be able to change it over to being more CPU bound than i/o > >bound and produce the performance improvments you're looking for. > > Except we're nowhere near being IO efficient. The vast difference between Postgres IO rates and dd shows this. I suspectthat's because we're not giving the OS a list of IO to perform while we're doing our thing, but that's just a guess. Uh, huh? The dd was ~321000 and the slowest uncached PG run from Robert's latest tests was 337312.554, based on my inbox history at least. I don't consider ~4-5% difference to be vast. > >The caveat to this is if you have multiple i/o *channels* (which it > >looks like you don't in this case) where you can parallelize across > >those channels by having multiple processes involved. > > Keep in mind that multiple processes is in no way a requirement for that. Async IO would do that, or even just requestingstuff from the OS before we need it. While I agree with this in principle, experience has shown that it doesn't tend to work out as well as we'd like with a single process. > > We only support > >multiple i/o channels today with tablespaces and we can't span tables > >across tablespaces. That's a problem when working with large data sets, > >but I'm hopeful that this work will eventually lead to a parallelized > >Append node that operates against a partitioned/inheirited table to work > >across multiple tablespaces. > > Until we can get a single seqscan close to dd performance, I fear worrying about tablespaces and IO channels is entirelypremature. I feel like one of us is misunderstanding the numbers, which is probably in part because they're a bit piecemeal over email, but the seqscan speed in this case looks pretty close to dd performance for this particular test, when things are uncached. Cached numbers are different, but that's not what we're discussing here, I don't think. Don't get me wrong- I've definitely seen cases where we're CPU bound because of complex filters, etc, but that doesn't seem to be the case here. Thanks! Stephen
On Wed, Jan 28, 2015 at 8:27 PM, Stephen Frost <sfrost@snowman.net> wrote: > I feel like one of us is misunderstanding the numbers, which is probably > in part because they're a bit piecemeal over email, but the seqscan > speed in this case looks pretty close to dd performance for this > particular test, when things are uncached. Cached numbers are > different, but that's not what we're discussing here, I don't think. > > Don't get me wrong- I've definitely seen cases where we're CPU bound > because of complex filters, etc, but that doesn't seem to be the case > here. To try to clarify a bit: What we've testing here is a function I wrote called parallel_count(regclass), which counts all the visible tuples in a named relation. That runs as fast as dd, and giving it extra workers or prefetching or the ability to read the relation with different I/O patterns never seems to speed anything up very much. The story with parallel sequential scan itself may well be different, since that has a lot more CPU overhead than a dumb-simple tuple-counter. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Robert Haas <robertmhaas@gmail.com> writes: > On Wed, Jan 28, 2015 at 9:12 AM, Thom Brown <thom@linux.com> wrote: >> On 28 January 2015 at 14:03, Robert Haas <robertmhaas@gmail.com> wrote: >>> The problem here, as I see it, is that we're flying blind. If there's >>> just one spindle, I think it's got to be right to read the relation >>> sequentially. But if there are multiple spindles, it might not be, >>> but it seems hard to predict what we should do. We don't know what >>> the RAID chunk size is or how many spindles there are, so any guess as >>> to how to chunk up the relation and divide up the work between workers >>> is just a shot in the dark. >> >> Can't the planner take effective_io_concurrency into account? > > Maybe. It's answering a somewhat the right question -- to tell us how > many parallel I/O channels we think we've got. But I'm not quite sure > what the to do with that information in this case. I mean, if we've > got effective_io_concurrency = 6, does that mean it's right to start > scans in 6 arbitrary places in the relation and hope that keeps all > the drives busy? That seems like throwing darts at the wall. We have > no idea which parts are on which underlying devices. Or maybe it mean > we should prefetch 24MB, on the assumption that the RAID stripe is > 4MB? That's definitely blind guesswork. > > Considering the email Amit just sent, it looks like on this machine, > regardless of what algorithm we used, the scan took between 3 minutes > and 5.5 minutes, and most of them took between 4 minutes and 5.5 > minutes. The results aren't very predictable, more workers don't > necessarily help, and it's not really clear that any algorithm we've > tried is clearly better than any other. I experimented with > prefetching a bit yesterday, too, and it was pretty much the same. > Some settings made it slightly faster. Others made it slower. Whee! I have been researching this topic long time ago. One notably fact is that active prefetching disables automatic readahead prefetching (by Linux kernel), which can occour in larger granularities than 8K. Automatic readahead prefetching occours when consecutive addresses are read, which may happen by a seqscan but also by "accident" through an indexscan in correlated cases. My consequence was to NOT prefetch seqscans, because OS does good enough without advice. Prefetching indexscan heap accesses is very valuable though, but you need to detect the accidential sequential accesses to not hurt your performance in correlated cases. In general I can give you the hint to not only focus on HDDs with their single spindle. A single SATA SSD scales up to 32 (31 on Linux) requests in parallel (without RAID or anything else). The difference in throughput is extreme for this type of storage device. While single spinning HDDs can only gain up to ~20% by NCQ, SATA SSDs can easily gain up to 700%. +1 for using effective_io_concurrency to tune for this, since prefetching random addresses is effectively a type of parallel I/O. Regards, Daniel -- MSc. Daniel Bausch Research Assistant (Computer Science) Technische Universität Darmstadt http://www.dvs.tu-darmstadt.de/staff/dbausch
>
> I have tried the tests again and found that I have forgotten to increase
> max_worker_processes due to which the data is so different. Basically
> at higher client count it is just scanning lesser number of blocks in
> fixed chunk approach. So today I again tried with changing
> max_worker_processes and found that there is not much difference in
> performance at higher client count. I will take some more data for
> both block_by_block and fixed_chunk approach and repost the data.
>
I have again taken the data and found that there is not much difference
Fixed-Chunks | ||||
No. of workers/Time (ms) | 0 | 8 | 16 | 32 |
Run-1 | 322822 | 245759 | 330097 | 330002 |
Run-2 | 275685 | 275428 | 301625 | 286251 |
Run-3 | 252129 | 244167 | 303494 | 278604 |
Run-4 | 252528 | 259273 | 250438 | 258636 |
Run-5 | 250612 | 242072 | 235384 | 265918 |
Block-By-Block | ||||
No. of workers/Time (ms) | 0 | 8 | 16 | 32 |
Run-1 | 323084 | 341950 | 338999 | 334100 |
Run-2 | 310968 | 349366 | 344272 | 322643 |
Run-3 | 250312 | 336227 | 346276 | 322274 |
Run-4 | 262744 | 314489 | 351652 | 325135 |
Run-5 | 265987 | 316260 | 342924 | 319200 |
OTOH, spreading the I/O across multiple files is not a good thing, if you don't have a RAID setup like that. With a single spindle, you'll just induce more seeks.On 01/28/2015 04:16 AM, Robert Haas wrote:On Tue, Jan 27, 2015 at 6:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:Now, when you did what I understand to be the same test on the same
machine, you got times ranging from 9.1 seconds to 35.4 seconds.
Clearly, there is some difference between our test setups. Moreover,
I'm kind of suspicious about whether your results are actually
physically possible. Even in the best case where you somehow had the
maximum possible amount of data - 64 GB on a 64 GB machine - cached,
leaving no space for cache duplication between PG and the OS and no
space for the operating system or postgres itself - the table is 120
GB, so you've got to read *at least* 56 GB from disk. Reading 56 GB
from disk in 9 seconds represents an I/O rate of >6 GB/s. I grant that
there could be some speedup from issuing I/O requests in parallel
instead of serially, but that is a 15x speedup over dd, so I am a
little suspicious that there is some problem with the test setup,
especially because I cannot reproduce the results.
So I thought about this a little more, and I realized after some
poking around that hydra's disk subsystem is actually six disks
configured in a software RAID5[1]. So one advantage of the
chunk-by-chunk approach you are proposing is that you might be able to
get all of the disks chugging away at once, because the data is
presumably striped across all of them. Reading one block at a time,
you'll never have more than 1 or 2 disks going, but if you do
sequential reads from a bunch of different places in the relation, you
might manage to get all 6. So that's something to think about.
One could imagine an algorithm like this: as long as there are more
1GB segments remaining than there are workers, each worker tries to
chug through a separate 1GB segment. When there are not enough 1GB
segments remaining for that to work, then they start ganging up on the
same segments. That way, you get the benefit of spreading out the I/O
across multiple files (and thus hopefully multiple members of the RAID
group) when the data is coming from disk, but you can still keep
everyone busy until the end, which will be important when the data is
all in-memory and you're just limited by CPU bandwidth.
Perhaps the OS is smart enough to read in large-enough chunks that the occasional seek doesn't hurt much. But then again, why isn't the OS smart enough to read in large-enough chunks to take advantage of the RAID even when you read just a single file?
Jeff Janes <jeff.janes@gmail.com> writes: > On Tue, Jan 27, 2015 at 11:08 PM, Heikki Linnakangas < > hlinnakangas@vmware.com> wrote: >> OTOH, spreading the I/O across multiple files is not a good thing, if you >> don't have a RAID setup like that. With a single spindle, you'll just >> induce more seeks. >> >> Perhaps the OS is smart enough to read in large-enough chunks that the >> occasional seek doesn't hurt much. But then again, why isn't the OS smart >> enough to read in large-enough chunks to take advantage of the RAID even >> when you read just a single file? > In my experience with RAID, it is smart enough to take advantage of that. > If the raid controller detects a sequential access pattern read, it > initiates a read ahead on each disk to pre-position the data it will need > (or at least, the behavior I observe is as-if it did that). But maybe if > the sequential read is a bunch of "random" reads from different processes > which just happen to add up to sequential, that confuses the algorithm? If seqscan detection is being done at the level of the RAID controller, I rather imagine that the controller would not know which process had initiated which read anyway. But if it's being done at the level of the kernel, it's a whole nother thing, and I bet it *would* matter. regards, tom lane
On Thu, Jan 29, 2015 at 11:40 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: >> In my experience with RAID, it is smart enough to take advantage of that. >> If the raid controller detects a sequential access pattern read, it >> initiates a read ahead on each disk to pre-position the data it will need >> (or at least, the behavior I observe is as-if it did that). But maybe if >> the sequential read is a bunch of "random" reads from different processes >> which just happen to add up to sequential, that confuses the algorithm? > > If seqscan detection is being done at the level of the RAID controller, > I rather imagine that the controller would not know which process had > initiated which read anyway. But if it's being done at the level of the > kernel, it's a whole nother thing, and I bet it *would* matter. That was my feeling too. On the machine that Amit and I have been using for testing, we can't find any really convincing evidence that it matters. I won't be a bit surprised if there are other systems where it does matter, but I don't know how to find them except to encourage other people to help test. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 1/28/15 7:27 PM, Stephen Frost wrote: > * Jim Nasby (Jim.Nasby@BlueTreble.com) wrote: >> >On 1/28/15 9:56 AM, Stephen Frost wrote: >>> > >Such i/o systems do exist, but a single RAID5 group over spinning rust >>> > >with a simple filter isn't going to cut it with a modern CPU- we're just >>> > >too darn efficient to end up i/o bound in that case. A more complex >>> > >filter might be able to change it over to being more CPU bound than i/o >>> > >bound and produce the performance improvments you're looking for. >> > >> >Except we're nowhere near being IO efficient. The vast difference between Postgres IO rates and dd shows this. I suspectthat's because we're not giving the OS a list of IO to perform while we're doing our thing, but that's just a guess. > Uh, huh? The dd was ~321000 and the slowest uncached PG run from > Robert's latest tests was 337312.554, based on my inbox history at > least. I don't consider ~4-5% difference to be vast. Sorry, I was speaking more generally than this specific test. In the past I've definitely seen SeqScan performance that wasan order of magnitude slower than what dd would do. This was an older version of Postgres and an older version of linux,running on an iSCSI SAN. My suspicion is that the added IO latency imposed by iSCSI is what was causing this, but that'sjust conjecture. I think Robert was saying that he hasn't been able to see this effect on their test server... that makes me think it's doingread-ahead on the OS level. But I suspect it's pretty touch and go to rely on that; I'd prefer we have some way to explicitlyget that behavior where we want it. -- Jim Nasby, Data Architect, Blue Treble Consulting Data in Trouble? Get it in Treble! http://BlueTreble.com
Daniel, * Daniel Bausch (bausch@dvs.tu-darmstadt.de) wrote: > I have been researching this topic long time ago. One notably fact is > that active prefetching disables automatic readahead prefetching (by > Linux kernel), which can occour in larger granularities than 8K. > Automatic readahead prefetching occours when consecutive addresses are > read, which may happen by a seqscan but also by "accident" through an > indexscan in correlated cases. That strikes me as a pretty good point to consider. > My consequence was to NOT prefetch seqscans, because OS does good enough > without advice. Prefetching indexscan heap accesses is very valuable > though, but you need to detect the accidential sequential accesses to > not hurt your performance in correlated cases. Seems like we might be able to do that, it's not that different from what we do with the bitmap scan case, we'd just look at the bitmap and see if there's long runs of 1's. > In general I can give you the hint to not only focus on HDDs with their > single spindle. A single SATA SSD scales up to 32 (31 on Linux) > requests in parallel (without RAID or anything else). The difference in > throughput is extreme for this type of storage device. While single > spinning HDDs can only gain up to ~20% by NCQ, SATA SSDs can easily gain > up to 700%. I definitely agree with the idea that we should be looking at SSD-based systems but I don't know if anyone happens to have easy access to server gear with SSDs. I've got an SSD in my laptop, but that's not really the same thing. Thanks! Stephen
This patch depends on https://commitfest.postgresql.org/3/22/
Hi David and others! David Fetter <david@fetter.org> writes: > On Tue, Jan 27, 2015 at 08:02:37AM +0100, Daniel Bausch wrote: >> >> Tom Lane <tgl@sss.pgh.pa.us> writes: >> >> >> Wait for first IO, issue second IO request >> >> Compute >> >> Already have second IO request, issue third >> >> ... >> > >> >> We'd be a lot less sensitive to IO latency. >> > >> > It would take about five minutes of coding to prove or disprove this: >> > stick a PrefetchBuffer call into heapgetpage() to launch a request for the >> > next page as soon as we've read the current one, and then see if that >> > makes any obvious performance difference. I'm not convinced that it will, >> > but if it did then we could think about how to make it work for real. >> >> Sorry for dropping in so late... >> >> I have done all this two years ago. For TPC-H Q8, Q9, Q17, Q20, and Q21 >> I see a speedup of ~100% when using IndexScan prefetching + Nested-Loops >> Look-Ahead (the outer loop!). >> (On SSD with 32 Pages Prefetch/Look-Ahead + Cold Page Cache / Small RAM) > > Would you be so kind as to pass along any patches (ideally applicable > to git master), tests, and specific measurements you made? Attached find my patches based on the old revision 36f4c7843cf3d201279855ed9a6ebc1deb3c9463 (Adjust cube.out expected output for new test queries.) I did not test applicability against HEAD by now. Disclaimer: This was just a proof-of-concept and so is poor implementation quality. Nevertheless, performance looked promising while it still needs a lot of extra rules for special cases, like detecting accidential sequential scans. General assumption is: no concurrency - a single query owning the machine. Here is a comparison using dbt3. Q8, Q9, Q17, Q20, and Q21 are significantly improved. | | baseline | indexscan | indexscan+nestloop | | | | patch 1+2 | patch 3 | |-----+------------+------------+--------------------| | Q1 | 76.124261 | 73.165161 | 76.323119 | | Q2 | 9.676956 | 11.211073 | 10.480668 | | Q3 | 36.836417 | 36.268022 | 36.837226 | | Q4 | 48.707501 | 64.2255 | 30.872218 | | Q5 | 59.371467 | 59.205048 | 58.646096 | | Q6 | 70.514214 | 73.021006 | 72.64643 | | Q7 | 63.667594 | 63.258499 | 62.758288 | | Q8 | 70.640973 | 33.144454 | 32.530732 | | Q9 | 446.630473 | 379.063773 | 219.926094 | | Q10 | 49.616125 | 49.244744 | 48.411664 | | Q11 | 6.122317 | 6.158616 | 6.160189 | | Q12 | 74.294292 | 87.780442 | 87.533936 | | Q13 | 32.37932 | 32.771938 | 33.483444 | | Q14 | 47.836053 | 48.093996 | 47.72221 | | Q15 | 139.350038 | 138.880208 | 138.681336 | | Q16 | 12.092429 | 12.120661 | 11.668971 | | Q17 | 9.346636 | 4.106042 | 4.018951 | | Q18 | 66.106875 | 123.754111 | 122.623193 | | Q19 | 22.750504 | 23.191532 | 22.34084 | | Q20 | 80.481986 | 29.906274 | 28.58106 | | Q21 | 396.897269 | 355.45988 | 214.44184 | | Q22 | 6.834841 | 6.600922 | 6.524032 | Regards, Daniel -- MSc. Daniel Bausch Research Assistant (Computer Science) Technische Universität Darmstadt http://www.dvs.tu-darmstadt.de/staff/dbausch >From 569398929d899100b769abfd919bc3383626ac9f Mon Sep 17 00:00:00 2001 From: Daniel Bausch <bausch@dvs.tu-darmstadt.de> Date: Tue, 22 Oct 2013 15:22:25 +0200 Subject: [PATCH 1/4] Quick proof-of-concept for indexscan prefetching This implements a prefetching queue of tuples whose tid is read ahead. Their block number is quickly checked for random properties (not current block and not the block prefetched last). Random reads are prefetched. Up to 32 tuples are considered by default. The tids are queued in a fixed ring buffer. The prefetching is implemented in the generic part of the index scan, so it applies to all access methods. --- src/backend/access/index/indexam.c | 96 ++++++++++++++++++++++++++++++++++++++ src/include/access/relscan.h | 12 +++++ 2 files changed, 108 insertions(+) diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index b878155..1c54ef5 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -251,6 +251,12 @@ index_beginscan(Relation heapRelation, scan->heapRelation = heapRelation; scan->xs_snapshot = snapshot; +#ifdef USE_PREFETCH + scan->xs_prefetch_head = scan->xs_prefetch_tail = -1; + scan->xs_last_prefetch = -1; + scan->xs_done = false; +#endif + return scan; } @@ -432,6 +438,55 @@ index_restrpos(IndexScanDesc scan) FunctionCall1(procedure, PointerGetDatum(scan)); } +static int +index_prefetch_queue_space(IndexScanDesc scan) +{ + if (scan->xs_prefetch_tail < 0) + return INDEXSCAN_PREFETCH_COUNT; + + Assert(scan->xs_prefetch_head >= 0); + + return (INDEXSCAN_PREFETCH_COUNT + - (scan->xs_prefetch_tail - scan->xs_prefetch_head + 1)) + % INDEXSCAN_PREFETCH_COUNT; +} + +/* makes copy of ItemPointerData */ +static bool +index_prefetch_queue_push(IndexScanDesc scan, ItemPointer tid) +{ + Assert(index_prefetch_queue_space(scan) > 0); + + if (scan->xs_prefetch_tail == -1) + scan->xs_prefetch_head = scan->xs_prefetch_tail = 0; + else + scan->xs_prefetch_tail = + (scan->xs_prefetch_tail + 1) % INDEXSCAN_PREFETCH_COUNT; + + scan->xs_prefetch_queue[scan->xs_prefetch_tail] = *tid; + + return true; +} + +static ItemPointer +index_prefetch_queue_pop(IndexScanDesc scan) +{ + ItemPointer res; + + if (scan->xs_prefetch_head < 0) + return NULL; + + res = &scan->xs_prefetch_queue[scan->xs_prefetch_head]; + + if (scan->xs_prefetch_head == scan->xs_prefetch_tail) + scan->xs_prefetch_head = scan->xs_prefetch_tail = -1; + else + scan->xs_prefetch_head = + (scan->xs_prefetch_head + 1) % INDEXSCAN_PREFETCH_COUNT; + + return res; +} + /* ---------------- * index_getnext_tid - get the next TID from a scan * @@ -444,12 +499,52 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) { FmgrInfo *procedure; bool found; + ItemPointer from_queue; + BlockNumber pf_block; SCAN_CHECKS; GET_SCAN_PROCEDURE(amgettuple); Assert(TransactionIdIsValid(RecentGlobalXmin)); +#ifdef USE_PREFETCH + while (!scan->xs_done && index_prefetch_queue_space(scan) > 0) { + /* + * The AM's amgettuple proc finds the next index entry matching the + * scan keys, and puts the TID into scan->xs_ctup.t_self. It should + * also set scan->xs_recheck and possibly scan->xs_itup, though we pay + * no attention to those fields here. + */ + found = DatumGetBool(FunctionCall2(procedure, + PointerGetDatum(scan), + Int32GetDatum(direction))); + if (found) + { + index_prefetch_queue_push(scan, &scan->xs_ctup.t_self); + pf_block = ItemPointerGetBlockNumber(&scan->xs_ctup.t_self); + /* prefetch only if not the current buffer and not exactly the + * previously prefetched buffer (heuristic random detection) + * because sequential read-ahead would be redundant */ + if ((!BufferIsValid(scan->xs_cbuf) || + pf_block != BufferGetBlockNumber(scan->xs_cbuf)) && + pf_block != scan->xs_last_prefetch) + { + PrefetchBuffer(scan->heapRelation, MAIN_FORKNUM, pf_block); + scan->xs_last_prefetch = pf_block; + } + } + else + scan->xs_done = true; + } + from_queue = index_prefetch_queue_pop(scan); + if (from_queue) + { + scan->xs_ctup.t_self = *from_queue; + found = true; + } + else + found = false; +#else /* * The AM's amgettuple proc finds the next index entry matching the scan * keys, and puts the TID into scan->xs_ctup.t_self. It should also set @@ -459,6 +554,7 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) found = DatumGetBool(FunctionCall2(procedure, PointerGetDatum(scan), Int32GetDatum(direction))); +#endif /* Reset kill flag immediately for safety */ scan->kill_prior_tuple = false; diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h index 3a86ca4..bccc1a4 100644 --- a/src/include/access/relscan.h +++ b/src/include/access/relscan.h @@ -93,6 +93,18 @@ typedef struct IndexScanDescData /* state data for traversing HOT chains in index_getnext */ bool xs_continue_hot; /* T if must keep walking HOT chain */ + +#ifdef USE_PREFETCH +# ifndef INDEXSCAN_PREFETCH_COUNT +# define INDEXSCAN_PREFETCH_COUNT 32 +# endif + /* prefetch queue - ringbuffer */ + ItemPointerData xs_prefetch_queue[INDEXSCAN_PREFETCH_COUNT]; + int xs_prefetch_head; + int xs_prefetch_tail; + BlockNumber xs_last_prefetch; + bool xs_done; +#endif } IndexScanDescData; /* Struct for heap-or-index scans of system tables */ -- 2.0.5 >From 7cb5839dd7751bcdcae6e4cbf69cfd24af10a694 Mon Sep 17 00:00:00 2001 From: Daniel Bausch <bausch@dvs.tu-darmstadt.de> Date: Wed, 23 Oct 2013 09:45:11 +0200 Subject: [PATCH 2/4] Fix index-only scan and rescan Prefetching heap data for index-only scans does not make any sense and it uses a different field (itup), nevertheless. Deactivate the prefetch logic for index-only scans. Reset xs_done and the queue on rescan, so we find tuples again. Remember last prefetch to detect correlation. --- src/backend/access/index/indexam.c | 85 +++++++++++++++++++++----------------- 1 file changed, 47 insertions(+), 38 deletions(-) diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index 1c54ef5..d8a4622 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -353,6 +353,12 @@ index_rescan(IndexScanDesc scan, scan->kill_prior_tuple = false; /* for safety */ +#ifdef USE_PREFETCH + /* I think, it does not hurt to remember xs_last_prefetch */ + scan->xs_prefetch_head = scan->xs_prefetch_tail = -1; + scan->xs_done = false; +#endif + FunctionCall5(procedure, PointerGetDatum(scan), PointerGetDatum(keys), @@ -508,7 +514,47 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) Assert(TransactionIdIsValid(RecentGlobalXmin)); #ifdef USE_PREFETCH - while (!scan->xs_done && index_prefetch_queue_space(scan) > 0) { + if (!scan->xs_want_itup) + { + while (!scan->xs_done && index_prefetch_queue_space(scan) > 0) { + /* + * The AM's amgettuple proc finds the next index entry matching + * the scan keys, and puts the TID into scan->xs_ctup.t_self. It + * should also set scan->xs_recheck and possibly scan->xs_itup, + * though we pay no attention to those fields here. + */ + found = DatumGetBool(FunctionCall2(procedure, + PointerGetDatum(scan), + Int32GetDatum(direction))); + if (found) + { + index_prefetch_queue_push(scan, &scan->xs_ctup.t_self); + pf_block = ItemPointerGetBlockNumber(&scan->xs_ctup.t_self); + /* prefetch only if not the current buffer and not exactly the + * previously prefetched buffer (heuristic random detection) + * because sequential read-ahead would be redundant */ + if ((!BufferIsValid(scan->xs_cbuf) || + pf_block != BufferGetBlockNumber(scan->xs_cbuf)) && + pf_block != scan->xs_last_prefetch) + { + PrefetchBuffer(scan->heapRelation, MAIN_FORKNUM, pf_block); + scan->xs_last_prefetch = pf_block; + } + } + else + scan->xs_done = true; + } + from_queue = index_prefetch_queue_pop(scan); + if (from_queue) + { + scan->xs_ctup.t_self = *from_queue; + found = true; + } + else + found = false; + } + else +#endif /* * The AM's amgettuple proc finds the next index entry matching the * scan keys, and puts the TID into scan->xs_ctup.t_self. It should @@ -518,43 +564,6 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) found = DatumGetBool(FunctionCall2(procedure, PointerGetDatum(scan), Int32GetDatum(direction))); - if (found) - { - index_prefetch_queue_push(scan, &scan->xs_ctup.t_self); - pf_block = ItemPointerGetBlockNumber(&scan->xs_ctup.t_self); - /* prefetch only if not the current buffer and not exactly the - * previously prefetched buffer (heuristic random detection) - * because sequential read-ahead would be redundant */ - if ((!BufferIsValid(scan->xs_cbuf) || - pf_block != BufferGetBlockNumber(scan->xs_cbuf)) && - pf_block != scan->xs_last_prefetch) - { - PrefetchBuffer(scan->heapRelation, MAIN_FORKNUM, pf_block); - scan->xs_last_prefetch = pf_block; - } - } - else - scan->xs_done = true; - } - from_queue = index_prefetch_queue_pop(scan); - if (from_queue) - { - scan->xs_ctup.t_self = *from_queue; - found = true; - } - else - found = false; -#else - /* - * The AM's amgettuple proc finds the next index entry matching the scan - * keys, and puts the TID into scan->xs_ctup.t_self. It should also set - * scan->xs_recheck and possibly scan->xs_itup, though we pay no attention - * to those fields here. - */ - found = DatumGetBool(FunctionCall2(procedure, - PointerGetDatum(scan), - Int32GetDatum(direction))); -#endif /* Reset kill flag immediately for safety */ scan->kill_prior_tuple = false; -- 2.0.5 >From d8b1533955e3471fb2eb6a030619dcbc258955a8 Mon Sep 17 00:00:00 2001 From: Daniel Bausch <bausch@dvs.tu-darmstadt.de> Date: Mon, 28 Oct 2013 10:43:16 +0100 Subject: [PATCH 3/4] First try on tuple look-ahead in nestloop Similarly to the prefetching logic just added to the index scan, look ahead tuples in the outer loop of a nested loop scan. For every tuple looked ahead issue an explicit request for prefetching to the inner plan. Modify the index scan to react on this request. --- src/backend/access/index/indexam.c | 81 +++++++++----- src/backend/executor/execProcnode.c | 36 +++++++ src/backend/executor/nodeIndexscan.c | 16 +++ src/backend/executor/nodeNestloop.c | 200 ++++++++++++++++++++++++++++++++++- src/include/access/genam.h | 4 + src/include/executor/executor.h | 3 + src/include/executor/nodeIndexscan.h | 1 + src/include/nodes/execnodes.h | 12 +++ 8 files changed, 323 insertions(+), 30 deletions(-) diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index d8a4622..5f44dec 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -493,6 +493,57 @@ index_prefetch_queue_pop(IndexScanDesc scan) return res; } +#ifdef USE_PREFETCH +int +index_prefetch(IndexScanDesc scan, int maxPrefetch, ScanDirection direction) +{ + FmgrInfo *procedure; + int numPrefetched = 0; + bool found; + BlockNumber pf_block; + FILE *logfile; + + GET_SCAN_PROCEDURE(amgettuple); + + while (numPrefetched < maxPrefetch && !scan->xs_done && + index_prefetch_queue_space(scan) > 0) + { + /* + * The AM's amgettuple proc finds the next index entry matching the + * scan keys, and puts the TID into scan->xs_ctup.t_self. It should + * also set scan->xs_recheck and possibly scan->xs_itup, though we pay + * no attention to those fields here. + */ + found = DatumGetBool(FunctionCall2(procedure, + PointerGetDatum(scan), + Int32GetDatum(direction))); + if (found) + { + index_prefetch_queue_push(scan, &scan->xs_ctup.t_self); + pf_block = ItemPointerGetBlockNumber(&scan->xs_ctup.t_self); + + /* + * Prefetch only if not the current buffer and not exactly the + * previously prefetched buffer (heuristic random detection) + * because sequential read-ahead would be redundant + */ + if ((!BufferIsValid(scan->xs_cbuf) || + pf_block != BufferGetBlockNumber(scan->xs_cbuf)) && + pf_block != scan->xs_last_prefetch) + { + PrefetchBuffer(scan->heapRelation, MAIN_FORKNUM, pf_block); + scan->xs_last_prefetch = pf_block; + numPrefetched++; + } + } + else + scan->xs_done = true; + } + + return numPrefetched; +} +#endif + /* ---------------- * index_getnext_tid - get the next TID from a scan * @@ -506,7 +557,6 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) FmgrInfo *procedure; bool found; ItemPointer from_queue; - BlockNumber pf_block; SCAN_CHECKS; GET_SCAN_PROCEDURE(amgettuple); @@ -516,34 +566,7 @@ index_getnext_tid(IndexScanDesc scan, ScanDirection direction) #ifdef USE_PREFETCH if (!scan->xs_want_itup) { - while (!scan->xs_done && index_prefetch_queue_space(scan) > 0) { - /* - * The AM's amgettuple proc finds the next index entry matching - * the scan keys, and puts the TID into scan->xs_ctup.t_self. It - * should also set scan->xs_recheck and possibly scan->xs_itup, - * though we pay no attention to those fields here. - */ - found = DatumGetBool(FunctionCall2(procedure, - PointerGetDatum(scan), - Int32GetDatum(direction))); - if (found) - { - index_prefetch_queue_push(scan, &scan->xs_ctup.t_self); - pf_block = ItemPointerGetBlockNumber(&scan->xs_ctup.t_self); - /* prefetch only if not the current buffer and not exactly the - * previously prefetched buffer (heuristic random detection) - * because sequential read-ahead would be redundant */ - if ((!BufferIsValid(scan->xs_cbuf) || - pf_block != BufferGetBlockNumber(scan->xs_cbuf)) && - pf_block != scan->xs_last_prefetch) - { - PrefetchBuffer(scan->heapRelation, MAIN_FORKNUM, pf_block); - scan->xs_last_prefetch = pf_block; - } - } - else - scan->xs_done = true; - } + index_prefetch(scan, INDEXSCAN_PREFETCH_COUNT, direction); from_queue = index_prefetch_queue_pop(scan); if (from_queue) { diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c index 76dd62f..a8f2c90 100644 --- a/src/backend/executor/execProcnode.c +++ b/src/backend/executor/execProcnode.c @@ -741,3 +741,39 @@ ExecEndNode(PlanState *node) break; } } + + +#ifdef USE_PREFETCH +/* ---------------------------------------------------------------- + * ExecPrefetchNode + * + * Request explicit prefetching from a subtree/node without + * actually forming a tuple. + * + * The node shall request at most 'maxPrefetch' pages being + * prefetched. + * + * The function returns how many pages have been requested. + * + * Calling this function for a type that does not support + * prefetching is not an error. It just returns 0 as if no + * prefetching was possible. + * ---------------------------------------------------------------- + */ +int +ExecPrefetchNode(PlanState *node, int maxPrefetch) +{ + if (node == NULL) + return 0; + + switch (nodeTag(node)) + { + case T_IndexScanState: + return ExecPrefetchIndexScan((IndexScanState *) node, + maxPrefetch); + + default: + return 0; + } +} +#endif diff --git a/src/backend/executor/nodeIndexscan.c b/src/backend/executor/nodeIndexscan.c index f1062f1..bab0e7a 100644 --- a/src/backend/executor/nodeIndexscan.c +++ b/src/backend/executor/nodeIndexscan.c @@ -192,6 +192,22 @@ ExecReScanIndexScan(IndexScanState *node) ExecScanReScan(&node->ss); } +#ifdef USE_PREFETCH +/* ---------------------------------------------------------------- + * ExecPrefetchIndexScan(node, maxPrefetch) + * + * Trigger prefetching of index scan without actually fetching + * a tuple. + * ---------------------------------------------------------------- + */ +int +ExecPrefetchIndexScan(IndexScanState *node, int maxPrefetch) +{ + return index_prefetch(node->iss_ScanDesc, maxPrefetch, + node->ss.ps.state->es_direction); +} +#endif + /* * ExecIndexEvalRuntimeKeys diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c index c7a08ed..21ad5f8 100644 --- a/src/backend/executor/nodeNestloop.c +++ b/src/backend/executor/nodeNestloop.c @@ -25,6 +25,90 @@ #include "executor/nodeNestloop.h" #include "utils/memutils.h" +#ifdef USE_PREFETCH +static int +NestLoopLookAheadQueueSpace(NestLoopState *node) +{ + if (node->nl_lookAheadQueueTail < 0) + return NESTLOOP_PREFETCH_COUNT; + + Assert(node->nl_lookAheadQueueHead >= 0); + + return (NESTLOOP_PREFETCH_COUNT + - (node->nl_lookAheadQueueTail - node->nl_lookAheadQueueHead + 1)) + % NESTLOOP_PREFETCH_COUNT; +} + +/* makes materialized copy of tuple table slot */ +static bool +NestLoopLookAheadQueuePush(NestLoopState *node, TupleTableSlot *tuple) +{ + TupleTableSlot **queueEntry; + + Assert(NestLoopLookAheadQueueSpace(node) > 0); + + if (node->nl_lookAheadQueueTail == -1) + node->nl_lookAheadQueueHead = node->nl_lookAheadQueueTail = 0; + else + node->nl_lookAheadQueueTail = + (node->nl_lookAheadQueueTail +1) % NESTLOOP_PREFETCH_COUNT; + + queueEntry = &node->nl_lookAheadQueue[node->nl_lookAheadQueueTail]; + + if (!(*queueEntry)) + { + *queueEntry = ExecInitExtraTupleSlot(node->js.ps.state); + ExecSetSlotDescriptor(*queueEntry, + ExecGetResultType(outerPlanState(node))); + } + + ExecCopySlot(*queueEntry, tuple); + + return true; +} + +static TupleTableSlot * +NestLoopLookAheadQueuePop(NestLoopState *node) +{ + TupleTableSlot *res; + + if (node->nl_lookAheadQueueHead < 0) + return NULL; + + res = node->nl_lookAheadQueue[node->nl_lookAheadQueueHead]; + + if (node->nl_lookAheadQueueHead == node->nl_lookAheadQueueTail) + node->nl_lookAheadQueueHead = node->nl_lookAheadQueueTail = -1; + else + node->nl_lookAheadQueueHead = + (node->nl_lookAheadQueueHead + 1) % NESTLOOP_PREFETCH_COUNT; + + return res; +} + +static void +NestLoopLookAheadQueueClear(NestLoopState *node) +{ + TupleTableSlot *lookAheadTuple; + int i; + + /* + * As we do not clear the tuple table slots on pop, we need to scan the + * whole array, regardless of the current queue fill. + * + * We cannot really free the slot, as there is no well defined interface + * for that, but the emptied slots will be freed when the query ends. + */ + for (i = 0; i < NESTLOOP_PREFETCH_COUNT; i++) + { + lookAheadTuple = node->nl_lookAheadQueue[i]; + /* look only on pointer - all non NULL fields are non-empty */ + if (lookAheadTuple) + ExecClearTuple(lookAheadTuple); + } + +} +#endif /* USE_PREFETCH */ /* ---------------------------------------------------------------- * ExecNestLoop(node) @@ -120,7 +204,87 @@ ExecNestLoop(NestLoopState *node) if (node->nl_NeedNewOuter) { ENL1_printf("getting new outer tuple"); - outerTupleSlot = ExecProcNode(outerPlan); + +#ifdef USE_PREFETCH + /* + * While we have outer tuples and were not able to request enought + * prefetching from the inner plan to properly load the system, + * request more outer tuples and inner prefetching for them. + * + * Unfortunately we can do outer look-ahead directed prefetching + * only when we are rescanning the inner plan anyway; otherwise we + * would break the inner scan. Only an independent copy of the + * inner plan state would allow us to prefetch accross inner loops + * regardless of inner scan position. + */ + while (!node->nl_lookAheadDone && + node->nl_numInnerPrefetched < NESTLOOP_PREFETCH_COUNT && + NestLoopLookAheadQueueSpace(node) > 0) + { + TupleTableSlot *lookAheadTupleSlot = ExecProcNode(outerPlan); + + if (!TupIsNull(lookAheadTupleSlot)) + { + NestLoopLookAheadQueuePush(node, lookAheadTupleSlot); + + /* + * Set inner params according to look-ahead tuple. + * + * Fetch the values of any outer Vars that must be passed + * to the inner scan, and store them in the appropriate + * PARAM_EXEC slots. + */ + foreach(lc, nl->nestParams) + { + NestLoopParam *nlp = (NestLoopParam *) lfirst(lc); + int paramno = nlp->paramno; + ParamExecData *prm; + + prm = &(econtext->ecxt_param_exec_vals[paramno]); + /* Param value should be an OUTER_VAR var */ + Assert(IsA(nlp->paramval, Var)); + Assert(nlp->paramval->varno == OUTER_VAR); + Assert(nlp->paramval->varattno > 0); + prm->value = slot_getattr(lookAheadTupleSlot, + nlp->paramval->varattno, + &(prm->isnull)); + /* Flag parameter value as changed */ + innerPlan->chgParam = + bms_add_member(innerPlan->chgParam, paramno); + } + + /* + * Rescan inner plan with changed parameters and request + * explicit prefetch. Limit the inner prefetch amount + * according to our own bookkeeping. + * + * When the so processed outer tuple gets finally active + * in the inner loop, the inner plan will autonomously + * prefetch the same tuples again. This is redundant but + * avoiding that seems too complicated for now. It should + * not hurt too much and may even help in case the + * prefetched blocks have been evicted again in the + * meantime. + */ + ExecReScan(innerPlan); + node->nl_numInnerPrefetched += + ExecPrefetchNode(innerPlan, + NESTLOOP_PREFETCH_COUNT - + node->nl_numInnerPrefetched); + } + else + node->nl_lookAheadDone = true; /* outer plan exhausted */ + } + + /* + * If there is already the next outerPlan in our look-ahead queue, + * get the next outer tuple from there, otherwise execute the + * outer plan. + */ + outerTupleSlot = NestLoopLookAheadQueuePop(node); + if (TupIsNull(outerTupleSlot) && !node->nl_lookAheadDone) +#endif /* USE_PREFETCH */ + outerTupleSlot = ExecProcNode(outerPlan); /* * if there are no more outer tuples, then the join is complete.. @@ -174,6 +338,18 @@ ExecNestLoop(NestLoopState *node) innerTupleSlot = ExecProcNode(innerPlan); econtext->ecxt_innertuple = innerTupleSlot; +#ifdef USE_PREFETCH + /* + * Decrement prefetch counter as we cosume inner tuples. We need to + * check for >0 because prefetching might not have happened for the + * consumed tuple, maybe because explicit prefetching is not supported + * by the inner plan or because the explicit prefetching requested by + * us is exhausted and the inner plan is doing it on its own now. + */ + if (node->nl_numInnerPrefetched > 0) + node->nl_numInnerPrefetched--; +#endif + if (TupIsNull(innerTupleSlot)) { ENL1_printf("no inner tuple, need new outer tuple"); @@ -296,6 +472,9 @@ NestLoopState * ExecInitNestLoop(NestLoop *node, EState *estate, int eflags) { NestLoopState *nlstate; +#ifdef USE_PREFETCH + int i; +#endif /* check for unsupported flags */ Assert(!(eflags & (EXEC_FLAG_BACKWARD | EXEC_FLAG_MARK))); @@ -381,6 +560,15 @@ ExecInitNestLoop(NestLoop *node, EState *estate, int eflags) nlstate->nl_NeedNewOuter = true; nlstate->nl_MatchedOuter = false; +#ifdef USE_PREFETCH + nlstate->nl_lookAheadQueueHead = nlstate->nl_lookAheadQueueTail = -1; + nlstate->nl_lookAheadDone = false; + nlstate->nl_numInnerPrefetched = 0; + + for (i = 0; i < NESTLOOP_PREFETCH_COUNT; i++) + nlstate->nl_lookAheadQueue[i] = NULL; +#endif + NL1_printf("ExecInitNestLoop: %s\n", "node initialized"); @@ -409,6 +597,10 @@ ExecEndNestLoop(NestLoopState *node) */ ExecClearTuple(node->js.ps.ps_ResultTupleSlot); +#ifdef USE_PREFETCH + NestLoopLookAheadQueueClear(node); +#endif + /* * close down subplans */ @@ -444,4 +636,10 @@ ExecReScanNestLoop(NestLoopState *node) node->js.ps.ps_TupFromTlist = false; node->nl_NeedNewOuter = true; node->nl_MatchedOuter = false; + +#ifdef USE_PREFETCH + NestLoopLookAheadQueueClear(node); + node->nl_lookAheadDone = false; + node->nl_numInnerPrefetched = 0; +#endif } diff --git a/src/include/access/genam.h b/src/include/access/genam.h index a800041..7733b3c 100644 --- a/src/include/access/genam.h +++ b/src/include/access/genam.h @@ -146,6 +146,10 @@ extern void index_markpos(IndexScanDesc scan); extern void index_restrpos(IndexScanDesc scan); extern ItemPointer index_getnext_tid(IndexScanDesc scan, ScanDirection direction); +#ifdef USE_PREFETCH +extern int index_prefetch(IndexScanDesc scan, int maxPrefetch, + ScanDirection direction); +#endif extern HeapTuple index_fetch_heap(IndexScanDesc scan); extern HeapTuple index_getnext(IndexScanDesc scan, ScanDirection direction); extern int64 index_getbitmap(IndexScanDesc scan, TIDBitmap *bitmap); diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index 75841c8..88d0522 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -221,6 +221,9 @@ extern PlanState *ExecInitNode(Plan *node, EState *estate, int eflags); extern TupleTableSlot *ExecProcNode(PlanState *node); extern Node *MultiExecProcNode(PlanState *node); extern void ExecEndNode(PlanState *node); +#ifdef USE_PREFETCH +extern int ExecPrefetchNode(PlanState *node, int maxPrefetch); +#endif /* * prototypes from functions in execQual.c diff --git a/src/include/executor/nodeIndexscan.h b/src/include/executor/nodeIndexscan.h index 71dbd9c..f93632c 100644 --- a/src/include/executor/nodeIndexscan.h +++ b/src/include/executor/nodeIndexscan.h @@ -18,6 +18,7 @@ extern IndexScanState *ExecInitIndexScan(IndexScan *node, EState *estate, int eflags); extern TupleTableSlot *ExecIndexScan(IndexScanState *node); +extern int ExecPrefetchIndexScan(IndexScanState *node, int maxPrefetch); extern void ExecEndIndexScan(IndexScanState *node); extern void ExecIndexMarkPos(IndexScanState *node); extern void ExecIndexRestrPos(IndexScanState *node); diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 3b430e0..27fe65d 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -1526,6 +1526,18 @@ typedef struct NestLoopState bool nl_NeedNewOuter; bool nl_MatchedOuter; TupleTableSlot *nl_NullInnerTupleSlot; + +#ifdef USE_PREFETCH +# ifndef NESTLOOP_PREFETCH_COUNT +# define NESTLOOP_PREFETCH_COUNT 32 +# endif + /* look-ahead queue (for prefetching) - ringbuffer */ + TupleTableSlot *nl_lookAheadQueue[NESTLOOP_PREFETCH_COUNT]; + int nl_lookAheadQueueHead; + int nl_lookAheadQueueTail; + bool nl_lookAheadDone; + int nl_numInnerPrefetched; +#endif } NestLoopState; /* ---------------- -- 2.0.5 >From a1fcab2d9d001505a5fc25accdca71e88148e4ff Mon Sep 17 00:00:00 2001 From: Daniel Bausch <bausch@dvs.tu-darmstadt.de> Date: Tue, 29 Oct 2013 16:41:09 +0100 Subject: [PATCH 4/4] Limit recursive prefetching for merge join Add switch facility to limit the prefetching of a subtree recursively. In a first try add support for some variants of merge join. Distribute the prefetch allowance evenly between outer and inner subplan. --- src/backend/access/index/indexam.c | 5 +++- src/backend/executor/execProcnode.c | 47 +++++++++++++++++++++++++++++++++++- src/backend/executor/nodeAgg.c | 10 ++++++++ src/backend/executor/nodeIndexscan.c | 18 ++++++++++++++ src/backend/executor/nodeMaterial.c | 14 +++++++++++ src/backend/executor/nodeMergejoin.c | 22 +++++++++++++++++ src/include/access/relscan.h | 1 + src/include/executor/executor.h | 1 + src/include/executor/nodeAgg.h | 3 +++ src/include/executor/nodeIndexscan.h | 3 +++ src/include/executor/nodeMaterial.h | 3 +++ src/include/executor/nodeMergejoin.h | 3 +++ src/include/nodes/execnodes.h | 6 +++++ 13 files changed, 134 insertions(+), 2 deletions(-) diff --git a/src/backend/access/index/indexam.c b/src/backend/access/index/indexam.c index 5f44dec..354bde6 100644 --- a/src/backend/access/index/indexam.c +++ b/src/backend/access/index/indexam.c @@ -255,6 +255,7 @@ index_beginscan(Relation heapRelation, scan->xs_prefetch_head = scan->xs_prefetch_tail = -1; scan->xs_last_prefetch = -1; scan->xs_done = false; + scan->xs_prefetch_limit = INDEXSCAN_PREFETCH_COUNT; #endif return scan; @@ -506,7 +507,9 @@ index_prefetch(IndexScanDesc scan, int maxPrefetch, ScanDirection direction) GET_SCAN_PROCEDURE(amgettuple); while (numPrefetched < maxPrefetch && !scan->xs_done && - index_prefetch_queue_space(scan) > 0) + index_prefetch_queue_space(scan) > 0 && + index_prefetch_queue_space(scan) > + (INDEXSCAN_PREFETCH_COUNT - scan->xs_prefetch_limit)) { /* * The AM's amgettuple proc finds the next index entry matching the diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c index a8f2c90..a14a0d0 100644 --- a/src/backend/executor/execProcnode.c +++ b/src/backend/executor/execProcnode.c @@ -745,6 +745,51 @@ ExecEndNode(PlanState *node) #ifdef USE_PREFETCH /* ---------------------------------------------------------------- + * ExecLimitPrefetchNode + * + * Limit the amount of prefetching that may be requested by + * a subplan. + * + * Most of the handlers just pass-through the received value + * to their subplans. That is the case, when they have just + * one subplan that might prefetch. If they have two subplans + * intelligent heuristics need to be applied to distribute the + * prefetch allowance in a way delivering overall advantage. + * ---------------------------------------------------------------- + */ +void +ExecLimitPrefetchNode(PlanState *node, int limit) +{ + if (node == NULL) + return; + + switch (nodeTag(node)) + { + case T_IndexScanState: + ExecLimitPrefetchIndexScan((IndexScanState *) node, limit); + break; + + case T_MergeJoinState: + ExecLimitPrefetchMergeJoin((MergeJoinState *) node, limit); + break; + + case T_MaterialState: + ExecLimitPrefetchMaterial((MaterialState *) node, limit); + break; + + case T_AggState: + ExecLimitPrefetchAgg((AggState *) node, limit); + break; + + default: + elog(INFO, + "missing ExecLimitPrefetchNode handler for node type: %d", + (int) nodeTag(node)); + break; + } +} + +/* ---------------------------------------------------------------- * ExecPrefetchNode * * Request explicit prefetching from a subtree/node without @@ -776,4 +821,4 @@ ExecPrefetchNode(PlanState *node, int maxPrefetch) return 0; } } -#endif +#endif /* USE_PREFETCH */ diff --git a/src/backend/executor/nodeAgg.c b/src/backend/executor/nodeAgg.c index e02a6ff..94f6d77 100644 --- a/src/backend/executor/nodeAgg.c +++ b/src/backend/executor/nodeAgg.c @@ -1877,6 +1877,16 @@ ExecInitAgg(Agg *node, EState *estate, int eflags) return aggstate; } +#ifdef USE_PREFETCH +void +ExecLimitPrefetchAgg(AggState *node, int limit) +{ + Assert(node != NULL); + + ExecLimitPrefetchNode(outerPlanState(node), limit); +} +#endif + static Datum GetAggInitVal(Datum textInitVal, Oid transtype) { diff --git a/src/backend/executor/nodeIndexscan.c b/src/backend/executor/nodeIndexscan.c index bab0e7a..6ea236e 100644 --- a/src/backend/executor/nodeIndexscan.c +++ b/src/backend/executor/nodeIndexscan.c @@ -640,6 +640,24 @@ ExecInitIndexScan(IndexScan *node, EState *estate, int eflags) return indexstate; } +#ifdef USE_PREFETCH +/* ---------------------------------------------------------------- + * ExecLimitPrefetchIndexScan + * + * Sets/changes the number of tuples whose pages to request in + * advance. + * ---------------------------------------------------------------- + */ +void +ExecLimitPrefetchIndexScan(IndexScanState *node, int limit) +{ + Assert(node != NULL); + Assert(node->iss_ScanDesc != NULL); + + node->iss_ScanDesc->xs_prefetch_limit = limit; +} +#endif + /* * ExecIndexBuildScanKeys diff --git a/src/backend/executor/nodeMaterial.c b/src/backend/executor/nodeMaterial.c index 7a82f56..3370362 100644 --- a/src/backend/executor/nodeMaterial.c +++ b/src/backend/executor/nodeMaterial.c @@ -232,6 +232,20 @@ ExecInitMaterial(Material *node, EState *estate, int eflags) return matstate; } +#ifdef USE_PREFETCH +/* ---------------------------------------------------------------- + * ExecLimitPrefetchMaterial + * ---------------------------------------------------------------- + */ +void +ExecLimitPrefetchMaterial(MaterialState *node, int limit) +{ + Assert(node != NULL); + + ExecLimitPrefetchNode(outerPlanState(node), limit); +} +#endif + /* ---------------------------------------------------------------- * ExecEndMaterial * ---------------------------------------------------------------- diff --git a/src/backend/executor/nodeMergejoin.c b/src/backend/executor/nodeMergejoin.c index e69bc64..f25e074 100644 --- a/src/backend/executor/nodeMergejoin.c +++ b/src/backend/executor/nodeMergejoin.c @@ -1627,6 +1627,10 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) mergestate->mj_OuterTupleSlot = NULL; mergestate->mj_InnerTupleSlot = NULL; +#ifdef USE_PREFETCH + ExecLimitPrefetchMergeJoin(mergestate, MERGEJOIN_PREFETCH_COUNT); +#endif + /* * initialization successful */ @@ -1636,6 +1640,24 @@ ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags) return mergestate; } +#ifdef USE_PREFETCH +/* ---------------------------------------------------------------- + * ExecLimitPrefetchMergeJoin + * ---------------------------------------------------------------- + */ +void +ExecLimitPrefetchMergeJoin(MergeJoinState *node, int limit) +{ + int outerLimit = limit/2; + int innerLimit = limit/2; + + Assert(node != NULL); + + ExecLimitPrefetchNode(outerPlanState(node), outerLimit); + ExecLimitPrefetchNode(innerPlanState(node), innerLimit); +} +#endif + /* ---------------------------------------------------------------- * ExecEndMergeJoin * diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h index bccc1a4..3297900 100644 --- a/src/include/access/relscan.h +++ b/src/include/access/relscan.h @@ -104,6 +104,7 @@ typedef struct IndexScanDescData int xs_prefetch_tail; BlockNumber xs_last_prefetch; bool xs_done; + int xs_prefetch_limit; #endif } IndexScanDescData; diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h index 88d0522..09b94e0 100644 --- a/src/include/executor/executor.h +++ b/src/include/executor/executor.h @@ -222,6 +222,7 @@ extern TupleTableSlot *ExecProcNode(PlanState *node); extern Node *MultiExecProcNode(PlanState *node); extern void ExecEndNode(PlanState *node); #ifdef USE_PREFETCH +extern void ExecLimitPrefetchNode(PlanState *node, int limit); extern int ExecPrefetchNode(PlanState *node, int maxPrefetch); #endif diff --git a/src/include/executor/nodeAgg.h b/src/include/executor/nodeAgg.h index 38823d6..f775ec8 100644 --- a/src/include/executor/nodeAgg.h +++ b/src/include/executor/nodeAgg.h @@ -17,6 +17,9 @@ #include "nodes/execnodes.h" extern AggState *ExecInitAgg(Agg *node, EState *estate, int eflags); +#ifdef USE_PREFETCH +extern void ExecLimitPrefetchAgg(AggState *node, int limit); +#endif extern TupleTableSlot *ExecAgg(AggState *node); extern void ExecEndAgg(AggState *node); extern void ExecReScanAgg(AggState *node); diff --git a/src/include/executor/nodeIndexscan.h b/src/include/executor/nodeIndexscan.h index f93632c..ccf3121 100644 --- a/src/include/executor/nodeIndexscan.h +++ b/src/include/executor/nodeIndexscan.h @@ -17,6 +17,9 @@ #include "nodes/execnodes.h" extern IndexScanState *ExecInitIndexScan(IndexScan *node, EState *estate, int eflags); +#ifdef USE_PREFETCH +extern void ExecLimitPrefetchIndexScan(IndexScanState *node, int limit); +#endif extern TupleTableSlot *ExecIndexScan(IndexScanState *node); extern int ExecPrefetchIndexScan(IndexScanState *node, int maxPrefetch); extern void ExecEndIndexScan(IndexScanState *node); diff --git a/src/include/executor/nodeMaterial.h b/src/include/executor/nodeMaterial.h index cfca0a5..5c81fe8 100644 --- a/src/include/executor/nodeMaterial.h +++ b/src/include/executor/nodeMaterial.h @@ -17,6 +17,9 @@ #include "nodes/execnodes.h" extern MaterialState *ExecInitMaterial(Material *node, EState *estate, int eflags); +#ifdef USE_PREFETCH +extern void ExecLimitPrefetchMaterial(MaterialState *node, int limit); +#endif extern TupleTableSlot *ExecMaterial(MaterialState *node); extern void ExecEndMaterial(MaterialState *node); extern void ExecMaterialMarkPos(MaterialState *node); diff --git a/src/include/executor/nodeMergejoin.h b/src/include/executor/nodeMergejoin.h index fa6b5e0..e402b42 100644 --- a/src/include/executor/nodeMergejoin.h +++ b/src/include/executor/nodeMergejoin.h @@ -17,6 +17,9 @@ #include "nodes/execnodes.h" extern MergeJoinState *ExecInitMergeJoin(MergeJoin *node, EState *estate, int eflags); +#ifdef USE_PREFETCH +extern void ExecLimitPrefetchMergeJoin(MergeJoinState *node, int limit); +#endif extern TupleTableSlot *ExecMergeJoin(MergeJoinState *node); extern void ExecEndMergeJoin(MergeJoinState *node); extern void ExecReScanMergeJoin(MergeJoinState *node); diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h index 27fe65d..64ed6fb 100644 --- a/src/include/nodes/execnodes.h +++ b/src/include/nodes/execnodes.h @@ -1585,6 +1585,12 @@ typedef struct MergeJoinState ExprContext *mj_InnerEContext; } MergeJoinState; +#ifdef USE_PREFETCH +# ifndef MERGEJOIN_PREFETCH_COUNT +# define MERGEJOIN_PREFETCH_COUNT 32 +# endif +#endif + /* ---------------- * HashJoinState information * -- 2.0.5
>
> On Thu, Jan 22, 2015 at 6:37 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
> >
> > (Please point out me if my understanding is incorrect.)
> >
> > What happen if dynamic background worker process tries to reference temporary
> > tables? Because buffer of temporary table blocks are allocated on private
> > address space, its recent status is not visible to other process unless it is
> > not flushed to the storage every time.
> >
> > Do we need to prohibit create_parallelscan_paths() to generate a path when
> > target relation is temporary one?
> >
>
> Yes, we need to prohibit parallel scans on temporary relations. Will fix.
>
Here is the latest patch which fixes reported issues and supported
Attachment
On Fri, Feb 6, 2015 at 9:43 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Here is the latest patch which fixes reported issues and supported > Prepared Statements and Explain Statement for parallel sequential > scan. > > The main purpose is to get the feedback if possible on overall > structure/design of code before I goahead. I'm not very happy with the way this is modularized: 1. The new parallel sequential scan node runs only in the master. The workers are running a regular sequential scan with a hack to make them scan a subset of the blocks. I think this is wrong; parallel sequential scan shouldn't require this kind of modifications to the non-parallel case. 2. InitiateWorkers() is entirely specific to the concerns of parallel sequential scan. After looking this over, I think there are three categories of things that need to be clearly separated. Some stuff is going to be needed for any parallel query; some stuff is going to be needed only for parallel scans but will be needed for any type of parallel scan, not just parallel sequential scan[1]; some stuff is needed for any type of node that returns tuples but not for nodes that don't return tuples (e.g. needed for ParallelSeqScan and ParallelHashJoin, but not needed for ParallelHash); and some stuff is only going to be needed for parallel sequential scan specifically. This patch mixes all of those concerns together in a single function. That won't do; this needs to be easily extensible to whatever someone wants to parallelize next. 3. I think the whole idea of using the portal infrastructure for this is wrong. We've talked about this before, but the fact that you're doing it this way is having a major impact on the whole design of the patch, and I can't believe it's ever going to be committable this way. To create a portal, you have to pretend that you received a protocol message, which you didn't; and you have to pretend there is an SQL query so you can call PortalDefineQuery. That's ugly. As far as I can see the only thing we really get out of any of that is that we can use the DestReceiver stuff to get the tuples back to the master, but that doesn't really work either, because you're having to hack printtup.c anyway. So from my point of view you're going through a bunch of layers that really don't have any value. Considering the way the parallel mode patch has evolved, I no longer think there's much point to passing anything other than raw tuples between the backends, so the whole idea of going through a deform/send/recv/form cycle seems like something we can entirely skip. 4. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company [1] It is of course arguable whether a parallel index-scan or parallel bitmap index-scan or parallel index-only-scan or parallel custom scan makes sense, but this patch shouldn't assume that we won't want to do those things. We have other places in the code that know about the concept of a scan as opposed to some other kind of executor construct, and we should preserve that distinction here.
On Fri, Feb 6, 2015 at 12:34 PM, Robert Haas <robertmhaas@gmail.com> wrote: > 4. Obviously that went out a bit too soon. Anyway, what I think we should do here is back up a bit and talk about what the problems are that we need to solve here and how each of them should be solved. I think there is some good code in this patch, but we really need to think about what the interfaces should look like and achieve a clean separation of concerns. Looking at the code for the non-parallel SeqScan node, there are basically two things going on here: 1. We call heap_getnext() to get the next tuple and store it into a TupleTableSlot. 2. Via ExecScan(), we do projection and apply quals. My first comment here is that I think we should actually teach heapam.c about parallelism. In other words, let's have an interface like this: extern Size heap_parallelscan_estimate(Snapshot snapshot); extern void heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation, Snapshot snapshot); extern HeapScanDesc heap_beginscan_parallel(ParallelHeapScanDesc); So the idea is that if you want to do a parallel scan, you call heap_parallelscan_estimate() to figure out how much space to reserve in your dynamic shared memory segment. Then you call heap_parallelscan_initialize() to initialize the chunk of memory once you have it. Then each backend that wants to assist in the parallel scan calls heap_beginscan_parallel() on that chunk of memory and gets its own HeapScanDesc. Then, they can all call heap_getnext() just as in the non-parallel case. The ParallelHeapScanDesc needs to contain the relation OID, the snapshot, the ending block number, and a current-block counter. Instead of automatically advancing to the next block, they use one of Andres's nifty new atomic ops to bump the current-block counter and then scan the block just before the new value. All this seems pretty straightforward, and if we decide to later change the way the relation gets scanned (e.g. in 1GB chunks rather than block-by-block) it can be handled here pretty easily. Now, let's suppose that we have this interface and for some reason we don't care about quals and projection - we just want to get the tuples back to the master. It's easy enough to create a parallel context that fires up a worker and lets the worker call heap_beginscan_parallel() and then heap_getnext() in a loop, but what does it do with the resulting tuples? We need a tuple queue that can be used to send the tuples back to master. That's also pretty easy: just set up a shm_mq and use shm_mq_send() to send each tuple. Use shm_mq_receive() in the master to read them back out. The only thing we need to be careful about is that the tuple descriptors match. It must be that they do, because the way the current parallel context patch works, the master is guaranteed to hold a lock on the relation from before the worker starts up until after it dies. But we could stash the tuple descriptor in shared memory and cross-check that it matches just to be sure. Anyway, this doesn't seem terribly complex although we might want to wrap some abstraction around it somehow so that every kind of parallelism that uses tuple queues can benefit from it. Perhaps this could even be built into the parallel context machinery somehow, or maybe it's something executor-specific. At any rate it looks simpler than what you've got now. The complicated part here seems to me to figure out what we need to pass from the parallel leader to the parallel worker to create enough state for quals and projection. If we want to be able to call ExecScan() without modification, which seems like a good goal, we're going to need a ScanState node, which is going to need to contain valid pointers to (at least) a ProjectionInfo, an ExprContext, and a List of quals. That in turn is going to require an ExecutorState. Serializing those things directly doesn't seem very practical; what we instead want to do is figure out what we can pass that will allow easy reconstruction of those data structures. Right now, you're passing the target list, the qual list, the range table, and the params, but the range table doesn't seem to be getting used anywhere. I wonder if we need it. If we could get away with just passing the target list and qual list, and params, we'd be doing pretty well, I think. But I'm not sure exactly what that looks like. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote: > The complicated part here seems to me to figure out what we need to > pass from the parallel leader to the parallel worker to create enough > state for quals and projection. If we want to be able to call > ExecScan() without modification, which seems like a good goal, we're > going to need a ScanState node, which is going to need to contain > valid pointers to (at least) a ProjectionInfo, an ExprContext, and a > List of quals. That in turn is going to require an ExecutorState. > Serializing those things directly doesn't seem very practical; what we > instead want to do is figure out what we can pass that will allow easy > reconstruction of those data structures. Right now, you're passing > the target list, the qual list, the range table, and the params, but > the range table doesn't seem to be getting used anywhere. I wonder if > we need it. If we could get away with just passing the target list > and qual list, and params, we'd be doing pretty well, I think. But > I'm not sure exactly what that looks like. IndexBuildHeapRangeScan shows how to do qual evaluation with relatively little setup: estate = CreateExecutorState(); econtext = GetPerTupleExprContext(estate); slot = MakeSingleTupleTableSlot(RelationGetDescr(heapRelation)); /* Arrange for econtext's scan tuple to be the tuple under test */ econtext->ecxt_scantuple = slot; /* Set up execution state for predicate, if any. */ predicate = (List *) ExecPrepareExpr((Expr *) indexInfo->ii_Predicate, estate); Then, for each tuple: ExecStoreTuple(heapTuple, slot, InvalidBuffer, false); And: if (!ExecQual(predicate, econtext, false)) continue; This looks like a good model to follow for parallel sequential scan. The point though is that I think we should do it directly rather than letting the portal machinery do it for us. Not sure how to get projection working yet. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote: > My first comment here is that I think we should actually teach > heapam.c about parallelism. I coded this up; see attached. I'm also attaching an updated version of the parallel count code revised to use this API. It's now called "parallel_count" rather than "parallel_dummy" and I removed some stupid stuff from it. I'm curious to see what other people think, but this seems much cleaner to me. With the old approach, the parallel-count code was duplicating some of the guts of heapam.c and dropping the rest on the floor; now it just asks for a parallel scan and away it goes. Similarly, if your parallel-seqscan patch wanted to scan block-by-block rather than splitting the relation into equal parts, or if it wanted to participate in the synchronized-seqcan stuff, there was no clean way to do that. With this approach, those decisions are - as they quite properly should be - isolated within heapam.c, rather than creeping into the executor. (These patches should be applied over parallel-mode-v4.patch.) -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
On 2015-02-06 22:57:43 -0500, Robert Haas wrote: > On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote: > > My first comment here is that I think we should actually teach > > heapam.c about parallelism. > > I coded this up; see attached. I'm also attaching an updated version > of the parallel count code revised to use this API. It's now called > "parallel_count" rather than "parallel_dummy" and I removed some > stupid stuff from it. I'm curious to see what other people think, but > this seems much cleaner to me. With the old approach, the > parallel-count code was duplicating some of the guts of heapam.c and > dropping the rest on the floor; now it just asks for a parallel scan > and away it goes. Similarly, if your parallel-seqscan patch wanted to > scan block-by-block rather than splitting the relation into equal > parts, or if it wanted to participate in the synchronized-seqcan > stuff, there was no clean way to do that. With this approach, those > decisions are - as they quite properly should be - isolated within > heapam.c, rather than creeping into the executor. I'm not convinced that that reasoning is generally valid. While it may work out nicely for seqscans - which might be useful enough on its own - the more stuff we parallelize the *more* the executor will have to know about it to make it sane. To actually scale nicely e.g. a parallel sort will have to execute the nodes below it on each backend, instead of doing that in one as a separate step, ferrying over all tuples to indivdual backends through queues, and only then parallezing the sort. Now. None of that is likely to matter immediately, but I think starting to build the infrastructure at the points where we'll later need it does make some sense. Greetings, Andres Freund -- Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
On Sat, Feb 7, 2015 at 4:30 PM, Andres Freund <andres@2ndquadrant.com> wrote: > On 2015-02-06 22:57:43 -0500, Robert Haas wrote: >> On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> > My first comment here is that I think we should actually teach >> > heapam.c about parallelism. >> >> I coded this up; see attached. I'm also attaching an updated version >> of the parallel count code revised to use this API. It's now called >> "parallel_count" rather than "parallel_dummy" and I removed some >> stupid stuff from it. I'm curious to see what other people think, but >> this seems much cleaner to me. With the old approach, the >> parallel-count code was duplicating some of the guts of heapam.c and >> dropping the rest on the floor; now it just asks for a parallel scan >> and away it goes. Similarly, if your parallel-seqscan patch wanted to >> scan block-by-block rather than splitting the relation into equal >> parts, or if it wanted to participate in the synchronized-seqcan >> stuff, there was no clean way to do that. With this approach, those >> decisions are - as they quite properly should be - isolated within >> heapam.c, rather than creeping into the executor. > > I'm not convinced that that reasoning is generally valid. While it may > work out nicely for seqscans - which might be useful enough on its own - > the more stuff we parallelize the *more* the executor will have to know > about it to make it sane. To actually scale nicely e.g. a parallel sort > will have to execute the nodes below it on each backend, instead of > doing that in one as a separate step, ferrying over all tuples to > indivdual backends through queues, and only then parallezing the > sort. > > Now. None of that is likely to matter immediately, but I think starting > to build the infrastructure at the points where we'll later need it does > make some sense. Well, I agree with you, but I'm not really sure what that has to do with the issue at hand. I mean, if we were to apply Amit's patch, we'd been in a situation where, for a non-parallel heap scan, heapam.c decides the order in which blocks get scanned, but for a parallel heap scan, nodeParallelSeqscan.c makes that decision. Maybe I'm an old fuddy-duddy[1] but that seems like an abstraction violation to me. I think the executor should see a parallel scan as a stream of tuples that streams into a bunch of backends in parallel, without really knowing how heapam.c is dividing up the work. That's how it's modularized today, and I don't see a reason to change it. Do you? Regarding tuple flow between backends, I've thought about that before, I agree that we need it, and I don't think I know how to do it. I can see how to have a group of processes executing a single node in parallel, or a single process executing a group of nodes we break off from the query tree and push down to it, but what you're talking about here is a group of processes executing a group of nodes jointly. That seems like an excellent idea, but I don't know how to design it. Actually routing the tuples between whichever backends we want to exchange them between is easy enough, but how do we decide whether to generate such a plan? What does the actual plan tree look like? Maybe we designate nodes as can-generate-multiple-tuple-streams (seq scan, mostly, I would think) and can-absorb-parallel-tuple-streams (sort, hash, materialize), or something like that, but I'm really fuzzy on the details. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company [1] Actually, there's not really any "maybe" about this.
>
> The complicated part here seems to me to figure out what we need to
> pass from the parallel leader to the parallel worker to create enough
> state for quals and projection. If we want to be able to call
> ExecScan() without modification, which seems like a good goal, we're
> going to need a ScanState node, which is going to need to contain
> valid pointers to (at least) a ProjectionInfo, an ExprContext, and a
> List of quals. That in turn is going to require an ExecutorState.
> Serializing those things directly doesn't seem very practical; what we
> instead want to do is figure out what we can pass that will allow easy
> reconstruction of those data structures. Right now, you're passing
> the target list, the qual list, the range table, and the params, but
> the range table doesn't seem to be getting used anywhere. I wonder if
> we need it.
>
> On Sat, Feb 7, 2015 at 4:30 PM, Andres Freund <andres@2ndquadrant.com> wrote:
> > On 2015-02-06 22:57:43 -0500, Robert Haas wrote:
> >> On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >> > My first comment here is that I think we should actually teach
> >> > heapam.c about parallelism.
> >>
> >> I coded this up; see attached. I'm also attaching an updated version
> >> of the parallel count code revised to use this API. It's now called
> >> "parallel_count" rather than "parallel_dummy" and I removed some
> >> stupid stuff from it. I'm curious to see what other people think, but
> >> this seems much cleaner to me. With the old approach, the
> >> parallel-count code was duplicating some of the guts of heapam.c and
> >> dropping the rest on the floor; now it just asks for a parallel scan
> >> and away it goes. Similarly, if your parallel-seqscan patch wanted to
> >> scan block-by-block rather than splitting the relation into equal
> >> parts, or if it wanted to participate in the synchronized-seqcan
> >> stuff, there was no clean way to do that. With this approach, those
> >> decisions are - as they quite properly should be - isolated within
> >> heapam.c, rather than creeping into the executor.
> >
> > I'm not convinced that that reasoning is generally valid. While it may
> > work out nicely for seqscans - which might be useful enough on its own -
> > the more stuff we parallelize the *more* the executor will have to know
> > about it to make it sane. To actually scale nicely e.g. a parallel sort
> > will have to execute the nodes below it on each backend, instead of
> > doing that in one as a separate step, ferrying over all tuples to
> > indivdual backends through queues, and only then parallezing the
> > sort.
> >
> > Now. None of that is likely to matter immediately, but I think starting
> > to build the infrastructure at the points where we'll later need it does
> > make some sense.
>
> Well, I agree with you, but I'm not really sure what that has to do
> with the issue at hand. I mean, if we were to apply Amit's patch,
> we'd been in a situation where, for a non-parallel heap scan, heapam.c
> decides the order in which blocks get scanned, but for a parallel heap
> scan, nodeParallelSeqscan.c makes that decision.
On Sat, Feb 7, 2015 at 10:36 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Well, I agree with you, but I'm not really sure what that has to do >> with the issue at hand. I mean, if we were to apply Amit's patch, >> we'd been in a situation where, for a non-parallel heap scan, heapam.c >> decides the order in which blocks get scanned, but for a parallel heap >> scan, nodeParallelSeqscan.c makes that decision. > > I think other places also decides about the order/way heapam.c has > to scan, example the order in which rows/pages has to traversed is > decided at portal/executor layer and the same is passed till heap and > in case of index, the scanlimits (heap_setscanlimits()) are decided > outside heapam.c and something similar is done for parallel seq scan. > In general, the scan is driven by Scandescriptor which is constructed > at upper level and there are some API's exposed to derive the scan. > If you are not happy with the current way nodeParallelSeqscan has > set the scan limits, we can have some form of callback which do the > required work and this callback can be called from heapam.c. I thought about a callback, but what's the benefit of doing that vs. hard-coding it in heapam.c? If the upper-layer wants to impose a TID qual or similar then heap_setscanlimits() makes sense, but that's effectively a filter condition, not a policy decision about the access pattern. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Feb 6, 2015 at 2:13 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> > The complicated part here seems to me to figure out what we need to
> > pass from the parallel leader to the parallel worker to create enough
> > state for quals and projection. If we want to be able to call
> > ExecScan() without modification, which seems like a good goal, we're
> > going to need a ScanState node, which is going to need to contain
> > valid pointers to (at least) a ProjectionInfo, an ExprContext, and a
> > List of quals. That in turn is going to require an ExecutorState.
> > Serializing those things directly doesn't seem very practical; what we
> > instead want to do is figure out what we can pass that will allow easy
> > reconstruction of those data structures. Right now, you're passing
> > the target list, the qual list, the range table, and the params, but
> > the range table doesn't seem to be getting used anywhere. I wonder if
> > we need it. If we could get away with just passing the target list
> > and qual list, and params, we'd be doing pretty well, I think. But
> > I'm not sure exactly what that looks like.
>
> IndexBuildHeapRangeScan shows how to do qual evaluation with
> relatively little setup:
>
>
> On Fri, Feb 6, 2015 at 9:43 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Here is the latest patch which fixes reported issues and supported
> > Prepared Statements and Explain Statement for parallel sequential
> > scan.
> >
> > The main purpose is to get the feedback if possible on overall
> > structure/design of code before I goahead.
>
>
> 2. InitiateWorkers() is entirely specific to the concerns of parallel
> sequential scan. After looking this over, I think there are three
> categories of things that need to be clearly separated. Some stuff is
> going to be needed for any parallel query; some stuff is going to be
> needed only for parallel scans but will be needed for any type of
> parallel scan, not just parallel sequential scan[1]; some stuff is
> needed for any type of node that returns tuples but not for nodes that
> don't return tuples (e.g. needed for ParallelSeqScan and
> ParallelHashJoin, but not needed for ParallelHash); and some stuff is
> only going to be needed for parallel sequential scan specifically.
> This patch mixes all of those concerns together in a single function.
> That won't do; this needs to be easily extensible to whatever someone
> wants to parallelize next.
>
>
> On Sat, Feb 7, 2015 at 10:36 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> Well, I agree with you, but I'm not really sure what that has to do
> >> with the issue at hand. I mean, if we were to apply Amit's patch,
> >> we'd been in a situation where, for a non-parallel heap scan, heapam.c
> >> decides the order in which blocks get scanned, but for a parallel heap
> >> scan, nodeParallelSeqscan.c makes that decision.
> >
> > I think other places also decides about the order/way heapam.c has
> > to scan, example the order in which rows/pages has to traversed is
> > decided at portal/executor layer and the same is passed till heap and
> > in case of index, the scanlimits (heap_setscanlimits()) are decided
> > outside heapam.c and something similar is done for parallel seq scan.
> > In general, the scan is driven by Scandescriptor which is constructed
> > at upper level and there are some API's exposed to derive the scan.
> > If you are not happy with the current way nodeParallelSeqscan has
> > set the scan limits, we can have some form of callback which do the
> > required work and this callback can be called from heapam.c.
>
> I thought about a callback, but what's the benefit of doing that vs.
> hard-coding it in heapam.c?
EnterpriseDB: http://www.enterprisedb.com
On Mon, Feb 9, 2015 at 2:31 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Another idea is to use Executor level interfaces (like ExecutorStart(), > ExecutorRun(), ExecutorEnd()) for execution rather than using Portal > level interfaces. I have used Portal level interfaces with the > thought that we can reuse the existing infrastructure of Portal to > make parallel execution of scrollable cursors, but as per my analysis > it is not so easy to support them especially backward scan, absolute/ > relative fetch, etc, so Executor level interfaces seems more appealing > to me (something like how Explain statement works (ExplainOnePlan)). > Using Executor level interfaces will have advantage that we can reuse them > for other parallel functionalaties. In this approach, we need to take > care of constructing relavant structures (with the information passed by > master backend) required for Executor interfaces, but I think these should > be lesser than what we need in previous approach (extract seqscan specific > stuff from executor). I think using the executor-level interfaces instead of the portal-level interfaces is a good idea. That would possibly let us altogether prohibit access to the portal layer from within a parallel worker, which seems like it might be a good sanity check to add. But that seems to still require us to have a PlannedStmt and a QueryDesc, and I'm not sure whether that's going to be too much of a pain. We might need to think about an alternative API for starting the Executor like ExecutorStartParallel() or ExecutorStartExtended(). But I'm not sure. If you can revise things to go through the executor interfaces I think that would be a good start, and then perhaps after that we can see what else makes sense to do. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 2015-02-07 17:16:12 -0500, Robert Haas wrote: > On Sat, Feb 7, 2015 at 4:30 PM, Andres Freund <andres@2ndquadrant.com> wrote: > > > [ criticicm of Amit's heapam integration ] > > I'm not convinced that that reasoning is generally valid. While it may > > work out nicely for seqscans - which might be useful enough on its own - > > the more stuff we parallelize the *more* the executor will have to know > > about it to make it sane. To actually scale nicely e.g. a parallel sort > > will have to execute the nodes below it on each backend, instead of > > doing that in one as a separate step, ferrying over all tuples to > > indivdual backends through queues, and only then parallezing the > > sort. > > > > Now. None of that is likely to matter immediately, but I think starting > > to build the infrastructure at the points where we'll later need it does > > make some sense. > > Well, I agree with you, but I'm not really sure what that has to do > with the issue at hand. I mean, if we were to apply Amit's patch, > we'd been in a situation where, for a non-parallel heap scan, heapam.c > decides the order in which blocks get scanned, but for a parallel heap > scan, nodeParallelSeqscan.c makes that decision. Maybe I'm an old > fuddy-duddy[1] but that seems like an abstraction violation to me. I > think the executor should see a parallel scan as a stream of tuples > that streams into a bunch of backends in parallel, without really > knowing how heapam.c is dividing up the work. That's how it's > modularized today, and I don't see a reason to change it. Do you? I don't really agree. Normally heapam just sequentially scan the heap in one go, not much logic to that. Ok, then there's also the synchronized seqscan stuff - which just about every user of heapscans but the executor promptly disables again. I don't think a heap_scan_page() or similar API will consitute a relevant layering violation over what we already have. Note that I'm not saying that Amit's patch is right - I haven't read it - but that I don't think a 'scan this range of pages' heapscan API would not be a bad idea. Not even just for parallelism, but for a bunch of usecases. > Regarding tuple flow between backends, I've thought about that before, > I agree that we need it, and I don't think I know how to do it. I can > see how to have a group of processes executing a single node in > parallel, or a single process executing a group of nodes we break off > from the query tree and push down to it, but what you're talking about > here is a group of processes executing a group of nodes jointly. I don't think it really is that. I think you'd do it essentially by introducing a couple more nodes. Something like SomeUpperLayerNode | | AggCombinerNode / \ / \ / \ PartialHashAggNode PartialHashAggNode .... .PartialHashAggNode... | | | | | | | | PartialSeqScan PartialSeqScan The only thing that'd potentially might need to end up working jointly jointly would be the block selection of the individual PartialSeqScans to avoid having to wait for stragglers for too long. E.g. each might just ask for a range of a 16 megabytes or so that it scans sequentially. In such a plan - a pretty sensible and not that uncommon thing for parallelized aggregates - you'd need to be able to tell the heap scans which blocks to scan. Right? > That seems like an excellent idea, but I don't know how to design it. > Actually routing the tuples between whichever backends we want to > exchange them between is easy enough, but how do we decide whether to > generate such a plan? What does the actual plan tree look like? I described above how I think it'd roughly look like. Whether to generate it probably would be dependant on the cardinality (not much point to do the above if all groups are distinct) and possibly the aggregates in use (if we have a parallizable sum/count/avg etc). > Maybe we designate nodes as can-generate-multiple-tuple-streams (seq > scan, mostly, I would think) and can-absorb-parallel-tuple-streams > (sort, hash, materialize), or something like that, but I'm really > fuzzy on the details. I don't think we really should have individual nodes that produce multiple streams - that seems like it'd end up being really complicated. I'd more say that we have distinct nodes (like the PartialSeqScan ones above) that do a teensy bit of coordination about which work to perform. Greetings, Andres Freund --Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
On Tue, Feb 10, 2015 at 2:48 AM, Andres Freund <andres@2ndquadrant.com> wrote: > Note that I'm not saying that Amit's patch is right - I haven't read it > - but that I don't think a 'scan this range of pages' heapscan API would > not be a bad idea. Not even just for parallelism, but for a bunch of > usecases. We do have that, already. heap_setscanlimits(). I'm just not convinced that that's the right way to split up a parallel scan. There's too much risk of ending up with a very-uneven distribution of work. >> Regarding tuple flow between backends, I've thought about that before, >> I agree that we need it, and I don't think I know how to do it. I can >> see how to have a group of processes executing a single node in >> parallel, or a single process executing a group of nodes we break off >> from the query tree and push down to it, but what you're talking about >> here is a group of processes executing a group of nodes jointly. > > I don't think it really is that. I think you'd do it essentially by > introducing a couple more nodes. Something like > > SomeUpperLayerNode > | > | > AggCombinerNode > / \ > / \ > / \ > PartialHashAggNode PartialHashAggNode .... .PartialHashAggNode ... > | | > | | > | | > | | > PartialSeqScan PartialSeqScan > > The only thing that'd potentially might need to end up working jointly > jointly would be the block selection of the individual PartialSeqScans > to avoid having to wait for stragglers for too long. E.g. each might > just ask for a range of a 16 megabytes or so that it scans sequentially. > > In such a plan - a pretty sensible and not that uncommon thing for > parallelized aggregates - you'd need to be able to tell the heap scans > which blocks to scan. Right? For this case, what I would imagine is that there is one parallel heap scan, and each PartialSeqScan attaches to it. The executor says "give me a tuple" and heapam.c provides one. Details like the chunk size are managed down inside heapam.c, and the executor does not know about them. It just knows that it can establish a parallel scan and then pull tuples from it. >> Maybe we designate nodes as can-generate-multiple-tuple-streams (seq >> scan, mostly, I would think) and can-absorb-parallel-tuple-streams >> (sort, hash, materialize), or something like that, but I'm really >> fuzzy on the details. > > I don't think we really should have individual nodes that produce > multiple streams - that seems like it'd end up being really > complicated. I'd more say that we have distinct nodes (like the > PartialSeqScan ones above) that do a teensy bit of coordination about > which work to perform. I think we're in violent agreement here, except for some terminological confusion. Are there N PartialSeqScan nodes, one running in each node, or is there one ParallelSeqScan node, which is copied and run jointly across N nodes? You can talk about either way and have it make sense, but we haven't had enough conversations about this on this list to have settled on a consistent set of vocabulary yet. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 2015-02-10 08:52:09 -0500, Robert Haas wrote: > On Tue, Feb 10, 2015 at 2:48 AM, Andres Freund <andres@2ndquadrant.com> wrote: > > Note that I'm not saying that Amit's patch is right - I haven't read it > > - but that I don't think a 'scan this range of pages' heapscan API would > > not be a bad idea. Not even just for parallelism, but for a bunch of > > usecases. > > We do have that, already. heap_setscanlimits(). I'm just not > convinced that that's the right way to split up a parallel scan. > There's too much risk of ending up with a very-uneven distribution of > work. If you make the chunks small enough, and then coordate only the chunk distribution, not really. > >> Regarding tuple flow between backends, I've thought about that before, > >> I agree that we need it, and I don't think I know how to do it. I can > >> see how to have a group of processes executing a single node in > >> parallel, or a single process executing a group of nodes we break off > >> from the query tree and push down to it, but what you're talking about > >> here is a group of processes executing a group of nodes jointly. > > > > I don't think it really is that. I think you'd do it essentially by > > introducing a couple more nodes. Something like > > > > SomeUpperLayerNode > > | > > | > > AggCombinerNode > > / \ > > / \ > > / \ > > PartialHashAggNode PartialHashAggNode .... .PartialHashAggNode ... > > | | > > | | > > | | > > | | > > PartialSeqScan PartialSeqScan > > > > The only thing that'd potentially might need to end up working jointly > > jointly would be the block selection of the individual PartialSeqScans > > to avoid having to wait for stragglers for too long. E.g. each might > > just ask for a range of a 16 megabytes or so that it scans sequentially. > > > > In such a plan - a pretty sensible and not that uncommon thing for > > parallelized aggregates - you'd need to be able to tell the heap scans > > which blocks to scan. Right? > > For this case, what I would imagine is that there is one parallel heap > scan, and each PartialSeqScan attaches to it. The executor says "give > me a tuple" and heapam.c provides one. Details like the chunk size > are managed down inside heapam.c, and the executor does not know about > them. It just knows that it can establish a parallel scan and then > pull tuples from it. I think that's a horrible approach that'll end up with far more entangled pieces than what you're trying to avoid. Unless the tuple flow is organized to only happen in the necessary cases the performance will be horrible. And good chunk sizes et al depend on higher layers, selectivity estimates and such. And that's planner/executor work, not the physical layer (which heapam.c pretty much is). A individual heap scan's state lives in process private memory. And if the results inside the separate workers should directly be used in the these workers without shipping over the network it'd be horrible to have the logic in the heapscan. How would you otherwise model an executor tree that does the seqscan and aggregation combined in multiple processes at the same time? > >> Maybe we designate nodes as can-generate-multiple-tuple-streams (seq > >> scan, mostly, I would think) and can-absorb-parallel-tuple-streams > >> (sort, hash, materialize), or something like that, but I'm really > >> fuzzy on the details. > > > > I don't think we really should have individual nodes that produce > > multiple streams - that seems like it'd end up being really > > complicated. I'd more say that we have distinct nodes (like the > > PartialSeqScan ones above) that do a teensy bit of coordination about > > which work to perform. > > I think we're in violent agreement here, except for some > terminological confusion. Are there N PartialSeqScan nodes, one > running in each node, or is there one ParallelSeqScan node, which is > copied and run jointly across N nodes? You can talk about either way > and have it make sense, but we haven't had enough conversations about > this on this list to have settled on a consistent set of vocabulary > yet. I pretty strongly believe that it has to be independent scan nodes. Both from a implementation and a conversational POV. They might have some very light cooperation between them (e.g. coordinating block ranges or such), but everything else should be separate. From an implementation POV it seems pretty awful to have executor node that's accessed by multiple separate backends - that'd mean it have to be concurrency safe, have state in shared memory and everything. Now, there'll be a node that needs to do some parallel magic - but in the above example that should be the AggCombinerNode, which would not only ask for tuples from one of the children at a time, but ask multiple ones in parallel. But even then it doesn't have to deal with concurrency around it's own state. Greetings, Andres Freund -- Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
On Tue, Feb 10, 2015 at 9:08 AM, Andres Freund <andres@2ndquadrant.com> wrote: > If you make the chunks small enough, and then coordate only the chunk > distribution, not really. True, but why do you want to do that in the executor instead of in the heapam? >> For this case, what I would imagine is that there is one parallel heap >> scan, and each PartialSeqScan attaches to it. The executor says "give >> me a tuple" and heapam.c provides one. Details like the chunk size >> are managed down inside heapam.c, and the executor does not know about >> them. It just knows that it can establish a parallel scan and then >> pull tuples from it. > > I think that's a horrible approach that'll end up with far more > entangled pieces than what you're trying to avoid. Unless the tuple flow > is organized to only happen in the necessary cases the performance will > be horrible. I can't understand this at all. A parallel heap scan, as I've coded it up, involves no tuple flow at all. All that's happening at the heapam.c layer is that we're coordinating which blocks to scan. Not to be disrespectful, but have you actually looked at the patch? > And good chunk sizes et al depend on higher layers, > selectivity estimates and such. And that's planner/executor work, not > the physical layer (which heapam.c pretty much is). If it's true that a good chunk size depends on the higher layers, then that would be a good argument for doing this differently, or at least exposing an API for the higher layers to tell heapam.c what chunk size they want. I hadn't considered that possibility - can you elaborate on why you think we might want to vary the chunk size? > A individual heap scan's state lives in process private memory. And if > the results inside the separate workers should directly be used in the > these workers without shipping over the network it'd be horrible to have > the logic in the heapscan. How would you otherwise model an executor > tree that does the seqscan and aggregation combined in multiple > processes at the same time? Again, the heap scan is not shipping anything anywhere ever in any design of any patch proposed or written. The results *are* directly used inside each individual worker. >> I think we're in violent agreement here, except for some >> terminological confusion. Are there N PartialSeqScan nodes, one >> running in each node, or is there one ParallelSeqScan node, which is >> copied and run jointly across N nodes? You can talk about either way >> and have it make sense, but we haven't had enough conversations about >> this on this list to have settled on a consistent set of vocabulary >> yet. > > I pretty strongly believe that it has to be independent scan nodes. Both > from a implementation and a conversational POV. They might have some > very light cooperation between them (e.g. coordinating block ranges or > such), but everything else should be separate. From an implementation > POV it seems pretty awful to have executor node that's accessed by > multiple separate backends - that'd mean it have to be concurrency safe, > have state in shared memory and everything. I don't agree with that, but again I think it's a terminological dispute. I think what will happen is that you will have a single node that gets copied into multiple backends, and in some cases a small portion of its state will live in shared memory. That's more or less what you're thinking of too, I think. But what I don't want is - if we've got a parallel scan-and-aggregate happening in N nodes, EXPLAIN shows N copies of all of that - not only because it's display clutter, but also because a plan to do that thing with 3 workers is fundamentally the same as a plan to do it with 30 workers. Those plans shouldn't look different, except perhaps for a line some place that says "Number of Workers: N". > Now, there'll be a node that needs to do some parallel magic - but in > the above example that should be the AggCombinerNode, which would not > only ask for tuples from one of the children at a time, but ask multiple > ones in parallel. But even then it doesn't have to deal with concurrency > around it's own state. Sure, we clearly want to minimize the amount of coordination between nodes. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 2015-02-10 09:23:02 -0500, Robert Haas wrote: > On Tue, Feb 10, 2015 at 9:08 AM, Andres Freund <andres@2ndquadrant.com> wrote: > > And good chunk sizes et al depend on higher layers, > > selectivity estimates and such. And that's planner/executor work, not > > the physical layer (which heapam.c pretty much is). > > If it's true that a good chunk size depends on the higher layers, then > that would be a good argument for doing this differently, or at least > exposing an API for the higher layers to tell heapam.c what chunk size > they want. I hadn't considered that possibility - can you elaborate > on why you think we might want to vary the chunk size? Because things like chunk size depend on the shape of the entire plan. If you have a 1TB table and want to sequentially scan it in parallel with 10 workers you better use some rather large chunks. That way readahead will be efficient in a cpu/socket local manner, i.e. directly reading in the pages into the directly connected memory of that cpu. Important for performance on a NUMA system, otherwise you'll constantly have everything go over the shared bus. But if you instead have a plan where the sequential scan goes over a 1GB table, perhaps with some relatively expensive filters, you'll really want a small chunks size to avoid waiting. The chunk size will also really depend on what other nodes are doing, at least if they can run in the same worker. Even without things like NUMA and readahead I'm pretty sure that you'll want a chunk size a good bit above one page. The locks we acquire for the buffercache lookup and for reading the page are already quite bad for performance/scalability; even if we don't always/often hit the same lock. Making 20 processes that scan pages in parallel acquire yet a another lock (that's shared between all of them!) for every single page won't be fun, especially without or fast filters. > >> For this case, what I would imagine is that there is one parallel heap > >> scan, and each PartialSeqScan attaches to it. The executor says "give > >> me a tuple" and heapam.c provides one. Details like the chunk size > >> are managed down inside heapam.c, and the executor does not know about > >> them. It just knows that it can establish a parallel scan and then > >> pull tuples from it. > > > > I think that's a horrible approach that'll end up with far more > > entangled pieces than what you're trying to avoid. Unless the tuple flow > > is organized to only happen in the necessary cases the performance will > > be horrible. > > I can't understand this at all. A parallel heap scan, as I've coded > it up, involves no tuple flow at all. All that's happening at the > heapam.c layer is that we're coordinating which blocks to scan. Not > to be disrespectful, but have you actually looked at the patch? No, and I said so upthread. I started commenting because you argued that architecturally parallelism belongs in heapam.c instead of upper layers, and I can't agree with that. I now have, and it looks less bad than I had assumed, sorry. Unfortunately I still think it's wrong approach, also sorry. As pointed out above (moved there after reading the patch...) I don't think a chunk size of 1 or any other constant size can make sense. I don't even believe it'll necessarily be constant across an entire query execution (big initially, small at the end). Now, we could move determining that before the query execution into executor initialization, but then we won't yet know how many workers we're going to get. We could add a function setting that at runtime, but that'd mix up responsibilities quite a bit. I also can't agree with having a static snapshot in shared memory put there by the initialization function. For one it's quite awkward to end up with several equivalent snapshots at various places in shared memory. Right now the entire query execution can share one snapshot, this way we'd end up with several of them. Imo for actual parallel query execution the plan should be shared once and then be reused for everything done in the name of the query. Without the need to do that you end up pretty much with only with setup for infrastructure so heap_parallelscan_nextpage is called. How about instead renaming heap_beginscan_internal() to _extended and offering an option to provide a callback + state that determines the next page? Additionally provide some separate functions managing a simple implementation of such a callback + state? Btw, using a atomic uint32 you'd end up without the spinlock and just about the same amount of code... Just do a atomic_fetch_add_until32(var, 1, InvalidBlockNumber)... ;) > >> I think we're in violent agreement here, except for some > >> terminological confusion. Are there N PartialSeqScan nodes, one > >> running in each node, or is there one ParallelSeqScan node, which is > >> copied and run jointly across N nodes? You can talk about either way > >> and have it make sense, but we haven't had enough conversations about > >> this on this list to have settled on a consistent set of vocabulary > >> yet. > > > > I pretty strongly believe that it has to be independent scan nodes. Both > > from a implementation and a conversational POV. They might have some > > very light cooperation between them (e.g. coordinating block ranges or > > such), but everything else should be separate. From an implementation > > POV it seems pretty awful to have executor node that's accessed by > > multiple separate backends - that'd mean it have to be concurrency safe, > > have state in shared memory and everything. > > I don't agree with that, but again I think it's a terminological > dispute. I think what will happen is that you will have a single node > that gets copied into multiple backends, and in some cases a small > portion of its state will live in shared memory. That's more or less > what you're thinking of too, I think. Well, let me put it that way, I think that the tuple flow has to be pretty much like I'd ascii-art'ed earlier. And that only very few nodes will need to coordinate between query execution happening in different workers. With that I mean it has to be possible to have queries like: ParallelismDrivingNode | ---------------- Parallelism boundary | NestLoop / \CSeqScan IndexScan Where the 'coordinated seqscan' scans a relation so that each tuple eventually gets returned once across all nodes, but the nested loop (and through it the index scan) will just run normally, without any coordination and parallelism. But everything below --- would happen multiple nodes. If you agree, yes, then we're in violent agreement ;). The "single node that gets copied" bit above makes me a bit unsure whether we are though. To me, given the existing executor code, it seems easiest to achieve that by having the ParallelismDrivingNode above having a dynamic number of nestloop children in different backends and point the coordinated seqscan to some shared state. As you point out, the number of these children cannot be certainly known (just targeted for) at plan time; that puts a certain limit on how independent they are. But since a large number of them can be independent between workers it seems awkward to generally treat them as being the same node across workers. But maybe that's just an issue with my mental model. > But what I don't want is - if we've got a parallel scan-and-aggregate > happening in N nodes, EXPLAIN shows N copies of all of that - not only > because it's display clutter, but also because a plan to do that thing > with 3 workers is fundamentally the same as a plan to do it with 30 > workers. Those plans shouldn't look different, except perhaps for a > line some place that says "Number of Workers: N". I'm really not concerned with what explain is going to show. We can do quite some fudging there - it's not like it's a 1:1 representation of the query plan. I think we're getting to the point where having a unique mapping from the plan to the execution tree is proving to be rather limiting anyway. Check for example discussion about join removal. But even for current code, showing only the custom plans for the first five EXPLAIN EXECUTEs is pretty nasty (Try explain that to somebody that doesn't know pg internals. Their looks are worth gold and can kill you at the same time) and should be done differently. And I actually can very well imagine that you'd want a option to show the different execution statistics for every worker in the ANALYZE case. Greetings, Andres Freund -- Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
On Tue, Feb 10, 2015 at 3:56 PM, Andres Freund <andres@2ndquadrant.com> wrote: > On 2015-02-10 09:23:02 -0500, Robert Haas wrote: >> On Tue, Feb 10, 2015 at 9:08 AM, Andres Freund <andres@2ndquadrant.com> wrote: >> > And good chunk sizes et al depend on higher layers, >> > selectivity estimates and such. And that's planner/executor work, not >> > the physical layer (which heapam.c pretty much is). >> >> If it's true that a good chunk size depends on the higher layers, then >> that would be a good argument for doing this differently, or at least >> exposing an API for the higher layers to tell heapam.c what chunk size >> they want. I hadn't considered that possibility - can you elaborate >> on why you think we might want to vary the chunk size? > > Because things like chunk size depend on the shape of the entire > plan. If you have a 1TB table and want to sequentially scan it in > parallel with 10 workers you better use some rather large chunks. That > way readahead will be efficient in a cpu/socket local manner, > i.e. directly reading in the pages into the directly connected memory of > that cpu. Important for performance on a NUMA system, otherwise you'll > constantly have everything go over the shared bus. But if you instead > have a plan where the sequential scan goes over a 1GB table, perhaps > with some relatively expensive filters, you'll really want a small > chunks size to avoid waiting. I see. That makes sense. > The chunk size will also really depend on > what other nodes are doing, at least if they can run in the same worker. Example? > Even without things like NUMA and readahead I'm pretty sure that you'll > want a chunk size a good bit above one page. The locks we acquire for > the buffercache lookup and for reading the page are already quite bad > for performance/scalability; even if we don't always/often hit the same > lock. Making 20 processes that scan pages in parallel acquire yet a > another lock (that's shared between all of them!) for every single page > won't be fun, especially without or fast filters. This is possible, but I'm skeptical. If the amount of other work we have to do that page is so little that the additional spinlock cycle per page causes meaningful contention, I doubt we should be parallelizing in the first place. > No, and I said so upthread. I started commenting because you argued that > architecturally parallelism belongs in heapam.c instead of upper layers, > and I can't agree with that. I now have, and it looks less bad than I > had assumed, sorry. OK, that's something. > Unfortunately I still think it's wrong approach, also sorry. > > As pointed out above (moved there after reading the patch...) I don't > think a chunk size of 1 or any other constant size can make sense. I > don't even believe it'll necessarily be constant across an entire query > execution (big initially, small at the end). Now, we could move > determining that before the query execution into executor > initialization, but then we won't yet know how many workers we're going > to get. We could add a function setting that at runtime, but that'd mix > up responsibilities quite a bit. I still think this belongs in heapam.c somehow or other. If the logic is all in the executor, then it becomes impossible for any code that doensn't use the executor to do a parallel heap scan, and that's probably bad. It's not hard to imagine something like CLUSTER wanting to reuse that code, and that won't be possible if the logic is up in some higher layer. If the logic we want is to start with a large chunk size and then switch to a small chunk size when there's not much of the relation left to scan, there's still no reason that can't be encapsulated in heapam.c. > Btw, using a atomic uint32 you'd end up without the spinlock and just > about the same amount of code... Just do a atomic_fetch_add_until32(var, > 1, InvalidBlockNumber)... ;) I thought of that, but I think there's an overflow hazard. > Where the 'coordinated seqscan' scans a relation so that each tuple > eventually gets returned once across all nodes, but the nested loop (and > through it the index scan) will just run normally, without any > coordination and parallelism. But everything below --- would happen > multiple nodes. If you agree, yes, then we're in violent agreement > ;). The "single node that gets copied" bit above makes me a bit unsure > whether we are though. Yeah, I think we're talking about the same thing. > To me, given the existing executor code, it seems easiest to achieve > that by having the ParallelismDrivingNode above having a dynamic number > of nestloop children in different backends and point the coordinated > seqscan to some shared state. As you point out, the number of these > children cannot be certainly known (just targeted for) at plan time; > that puts a certain limit on how independent they are. But since a > large number of them can be independent between workers it seems awkward > to generally treat them as being the same node across workers. But maybe > that's just an issue with my mental model. I think it makes sense to think of a set of tasks in which workers can assist. So you a query tree which is just one query tree, with no copies of the nodes, and then there are certain places in that query tree where a worker can jump in and assist that node. To do that, it will have a copy of the node, but that doesn't mean that all of the stuff inside the node becomes shared data at the code level, because that would be stupid. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Tue, Feb 10, 2015 at 3:56 PM, Andres Freund <andres@2ndquadrant.com> wrote:
> > On 2015-02-10 09:23:02 -0500, Robert Haas wrote:
> >> On Tue, Feb 10, 2015 at 9:08 AM, Andres Freund <andres@2ndquadrant.com> wrote:
> >
> > As pointed out above (moved there after reading the patch...) I don't
> > think a chunk size of 1 or any other constant size can make sense. I
> > don't even believe it'll necessarily be constant across an entire query
> > execution (big initially, small at the end). Now, we could move
> > determining that before the query execution into executor
> > initialization, but then we won't yet know how many workers we're going
> > to get. We could add a function setting that at runtime, but that'd mix
> > up responsibilities quite a bit.
>
> I still think this belongs in heapam.c somehow or other. If the logic
> is all in the executor, then it becomes impossible for any code that
> doensn't use the executor to do a parallel heap scan, and that's
> probably bad. It's not hard to imagine something like CLUSTER wanting
> to reuse that code, and that won't be possible if the logic is up in
> some higher layer. If the logic we want is to start with a large
> chunk size and then switch to a small chunk size when there's not much
> of the relation left to scan, there's still no reason that can't be
> encapsulated in heapam.c.
>
> > Btw, using a atomic uint32 you'd end up without the spinlock and just
> > about the same amount of code... Just do a atomic_fetch_add_until32(var,
> > 1, InvalidBlockNumber)... ;)
>
> I thought of that, but I think there's an overflow hazard.
>
> > Where the 'coordinated seqscan' scans a relation so that each tuple
> > eventually gets returned once across all nodes, but the nested loop (and
> > through it the index scan) will just run normally, without any
> > coordination and parallelism. But everything below --- would happen
> > multiple nodes. If you agree, yes, then we're in violent agreement
> > ;). The "single node that gets copied" bit above makes me a bit unsure
> > whether we are though.
>
> Yeah, I think we're talking about the same thing.
>
> > To me, given the existing executor code, it seems easiest to achieve
> > that by having the ParallelismDrivingNode above having a dynamic number
> > of nestloop children in different backends and point the coordinated
> > seqscan to some shared state. As you point out, the number of these
> > children cannot be certainly known (just targeted for) at plan time;
> > that puts a certain limit on how independent they are. But since a
> > large number of them can be independent between workers it seems awkward
> > to generally treat them as being the same node across workers. But maybe
> > that's just an issue with my mental model.
>
> I think it makes sense to think of a set of tasks in which workers can
> assist. So you a query tree which is just one query tree, with no
> copies of the nodes, and then there are certain places in that query
> tree where a worker can jump in and assist that node. To do that, it
> will have a copy of the node, but that doesn't mean that all of the
> stuff inside the node becomes shared data at the code level, because
> that would be stupid.
>
>
> On Mon, Feb 9, 2015 at 2:31 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Another idea is to use Executor level interfaces (like ExecutorStart(),
> > ExecutorRun(), ExecutorEnd()) for execution rather than using Portal
> > level interfaces. I have used Portal level interfaces with the
> > thought that we can reuse the existing infrastructure of Portal to
> > make parallel execution of scrollable cursors, but as per my analysis
> > it is not so easy to support them especially backward scan, absolute/
> > relative fetch, etc, so Executor level interfaces seems more appealing
> > to me (something like how Explain statement works (ExplainOnePlan)).
> > Using Executor level interfaces will have advantage that we can reuse them
> > for other parallel functionalaties. In this approach, we need to take
> > care of constructing relavant structures (with the information passed by
> > master backend) required for Executor interfaces, but I think these should
> > be lesser than what we need in previous approach (extract seqscan specific
> > stuff from executor).
>
> I think using the executor-level interfaces instead of the
> portal-level interfaces is a good idea. That would possibly let us
> altogether prohibit access to the portal layer from within a parallel
> worker, which seems like it might be a good sanity check to add. But
> that seems to still require us to have a PlannedStmt and a QueryDesc,
> and I'm not sure whether that's going to be too much of a pain. We
> might need to think about an alternative API for starting the Executor
> like ExecutorStartParallel() or ExecutorStartExtended(). But I'm not
> sure. If you can revise things to go through the executor interfaces
> I think that would be a good start, and then perhaps after that we can
> see what else makes sense to do.
>
Okay, I have modified the patch to use Executor level interfaces
Attachment
On 2015-02-11 15:49:17 -0500, Robert Haas wrote: > On Tue, Feb 10, 2015 at 3:56 PM, Andres Freund <andres@2ndquadrant.com> wrote: > >> On Tue, Feb 10, 2015 at 9:08 AM, Andres Freund <andres@2ndquadrant.com> wrote: > >> > And good chunk sizes et al depend on higher layers, > >> > selectivity estimates and such. And that's planner/executor work, not > >> > the physical layer (which heapam.c pretty much is). > >> > >> If it's true that a good chunk size depends on the higher layers, then > >> that would be a good argument for doing this differently, or at least > >> exposing an API for the higher layers to tell heapam.c what chunk size > >> they want. I hadn't considered that possibility - can you elaborate > >> on why you think we might want to vary the chunk size? > > > > Because things like chunk size depend on the shape of the entire > > plan. If you have a 1TB table and want to sequentially scan it in > > parallel with 10 workers you better use some rather large chunks. That > > way readahead will be efficient in a cpu/socket local manner, > > i.e. directly reading in the pages into the directly connected memory of > > that cpu. Important for performance on a NUMA system, otherwise you'll > > constantly have everything go over the shared bus. But if you instead > > have a plan where the sequential scan goes over a 1GB table, perhaps > > with some relatively expensive filters, you'll really want a small > > chunks size to avoid waiting. > > I see. That makes sense. > > > The chunk size will also really depend on > > what other nodes are doing, at least if they can run in the same worker. > > Example? A query whose runetime is dominated by a sequential scan (+ attached filter) is certainly going to require a bigger prefetch size than one that does other expensive stuff. Imagine parallelizing SELECT * FROM largetable WHERE col = low_cardinality_value; and SELECT * FROM largetable JOIN gigantic_table ON (index_nestloop_condition) WHERE col = high_cardinality_value; The first query will be a simple sequential and disk reads on largetable will be the major cost of executing it. In contrast the second query might very well sensibly be planned as a parallel sequential scan with the nested loop executing in the same worker. But the cost of the sequential scan itself will likely be completely drowned out by the nestloop execution - index probes are expensive/unpredictable. My guess is that the batch size can wil have to be computed based on the fraction of cost of the parallized work it has. > > Even without things like NUMA and readahead I'm pretty sure that you'll > > want a chunk size a good bit above one page. The locks we acquire for > > the buffercache lookup and for reading the page are already quite bad > > for performance/scalability; even if we don't always/often hit the same > > lock. Making 20 processes that scan pages in parallel acquire yet a > > another lock (that's shared between all of them!) for every single page > > won't be fun, especially without or fast filters. > > This is possible, but I'm skeptical. If the amount of other work we > have to do that page is so little that the additional spinlock cycle > per page causes meaningful contention, I doubt we should be > parallelizing in the first place. It's easy to see contention of buffer mapping (many workloads), buffer content and buffer header (especially btree roots and small foreign key target tables) locks. And for most of them we already avoid acquiring the same spinlock in all backends. Right now to process a page in a sequential scan we acquire a nonblocking buffer mapping lock (which doesn't use a spinlock anymore *because* it proved to be a bottleneck), a nonblocking content lock and a the buffer header spinlock. All of those are essentially partitioned - another spinlock shared between all workers will show up. > > As pointed out above (moved there after reading the patch...) I don't > > think a chunk size of 1 or any other constant size can make sense. I > > don't even believe it'll necessarily be constant across an entire query > > execution (big initially, small at the end). Now, we could move > > determining that before the query execution into executor > > initialization, but then we won't yet know how many workers we're going > > to get. We could add a function setting that at runtime, but that'd mix > > up responsibilities quite a bit. > > I still think this belongs in heapam.c somehow or other. If the logic > is all in the executor, then it becomes impossible for any code that > doensn't use the executor to do a parallel heap scan, and that's > probably bad. It's not hard to imagine something like CLUSTER wanting > to reuse that code, and that won't be possible if the logic is up in > some higher layer. Yea. > If the logic we want is to start with a large chunk size and then > switch to a small chunk size when there's not much of the relation > left to scan, there's still no reason that can't be encapsulated in > heapam.c. I don't mind having some logic in there, but I think you put in too much. The snapshot stuff should imo go, and the next page logic should be caller provided. > > Btw, using a atomic uint32 you'd end up without the spinlock and just > > about the same amount of code... Just do a atomic_fetch_add_until32(var, > > 1, InvalidBlockNumber)... ;) > > I thought of that, but I think there's an overflow hazard. That's why I said atomic_fetch_add_until32 - which can't overflow ;). I now remember that that was actually pulled on Heikki's request from the commited patch until a user shows up, but we can easily add it back. compare/exchange makes such things simple luckily. > > To me, given the existing executor code, it seems easiest to achieve > > that by having the ParallelismDrivingNode above having a dynamic number > > of nestloop children in different backends and point the coordinated > > seqscan to some shared state. As you point out, the number of these > > children cannot be certainly known (just targeted for) at plan time; > > that puts a certain limit on how independent they are. But since a > > large number of them can be independent between workers it seems awkward > > to generally treat them as being the same node across workers. But maybe > > that's just an issue with my mental model. > > I think it makes sense to think of a set of tasks in which workers can > assist. So you a query tree which is just one query tree, with no > copies of the nodes, and then there are certain places in that query > tree where a worker can jump in and assist that node. To do that, it > will have a copy of the node, but that doesn't mean that all of the > stuff inside the node becomes shared data at the code level, because > that would be stupid. My only "problem" with that description is that I think workers will have to work on more than one node - it'll be entire subtrees of the executor tree. Greetings, Andres Freund -- Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
>
> On 2015-02-11 15:49:17 -0500, Robert Haas wrote:
>
> A query whose runetime is dominated by a sequential scan (+ attached
> filter) is certainly going to require a bigger prefetch size than one
> that does other expensive stuff.
>
> Imagine parallelizing
> SELECT * FROM largetable WHERE col = low_cardinality_value;
> and
> SELECT *
> FROM largetable JOIN gigantic_table ON (index_nestloop_condition)
> WHERE col = high_cardinality_value;
>
> The first query will be a simple sequential and disk reads on largetable
> will be the major cost of executing it. In contrast the second query
> might very well sensibly be planned as a parallel sequential scan with
> the nested loop executing in the same worker. But the cost of the
> sequential scan itself will likely be completely drowned out by the
> nestloop execution - index probes are expensive/unpredictable.
>
> > I think it makes sense to think of a set of tasks in which workers can
> > assist. So you a query tree which is just one query tree, with no
> > copies of the nodes, and then there are certain places in that query
> > tree where a worker can jump in and assist that node. To do that, it
> > will have a copy of the node, but that doesn't mean that all of the
> > stuff inside the node becomes shared data at the code level, because
> > that would be stupid.
>
> My only "problem" with that description is that I think workers will
> have to work on more than one node - it'll be entire subtrees of the
> executor tree.
>
On 2015-02-18 16:59:26 +0530, Amit Kapila wrote: > On Tue, Feb 17, 2015 at 9:52 PM, Andres Freund <andres@2ndquadrant.com> > wrote: > > A query whose runetime is dominated by a sequential scan (+ attached > > filter) is certainly going to require a bigger prefetch size than one > > that does other expensive stuff. > > > > Imagine parallelizing > > SELECT * FROM largetable WHERE col = low_cardinality_value; > > and > > SELECT * > > FROM largetable JOIN gigantic_table ON (index_nestloop_condition) > > WHERE col = high_cardinality_value; > > > > The first query will be a simple sequential and disk reads on largetable > > will be the major cost of executing it. In contrast the second query > > might very well sensibly be planned as a parallel sequential scan with > > the nested loop executing in the same worker. But the cost of the > > sequential scan itself will likely be completely drowned out by the > > nestloop execution - index probes are expensive/unpredictable. > I think the work/task given to each worker should be as granular > as possible to make it more predictable. > I think the better way to parallelize such a work (Join query) is that > first worker does sequential scan and filtering on large table and > then pass it to next worker for doing join with gigantic_table. I'm pretty sure that'll result in rather horrible performance. IPC is rather expensive, you want to do as little of it as possible. > > > > > > I think it makes sense to think of a set of tasks in which workers can > > > assist. So you a query tree which is just one query tree, with no > > > copies of the nodes, and then there are certain places in that query > > > tree where a worker can jump in and assist that node. To do that, it > > > will have a copy of the node, but that doesn't mean that all of the > > > stuff inside the node becomes shared data at the code level, because > > > that would be stupid. > > > > My only "problem" with that description is that I think workers will > > have to work on more than one node - it'll be entire subtrees of the > > executor tree. > There could be some cases where it could be beneficial for worker > to process a sub-tree, but I think there will be more cases where > it will just work on a part of node and send the result back to either > master backend or another worker for further processing. I think many parallelism projects start out that way, and then notice that it doesn't parallelize very efficiently. The most extreme example, but common, is aggregation over large amounts of data - unless you want to ship huge amounts of data between processes eto parallize it you have to do the sequential scan and the pre-aggregate step (that e.g. selects count() and sum() to implement a avg over all the workers) inside one worker. Greetings, Andres Freund -- Andres Freund http://www.2ndQuadrant.com/PostgreSQL Development, 24x7 Support, Training & Services
> On 2015-02-18 16:59:26 +0530, Amit Kapila wrote:
>
> > There could be some cases where it could be beneficial for worker
> > to process a sub-tree, but I think there will be more cases where
> > it will just work on a part of node and send the result back to either
> > master backend or another worker for further processing.
>
> I think many parallelism projects start out that way, and then notice
> that it doesn't parallelize very efficiently.
>
> The most extreme example, but common, is aggregation over large amounts
> of data - unless you want to ship huge amounts of data between processes
> eto parallize it you have to do the sequential scan and the
> pre-aggregate step (that e.g. selects count() and sum() to implement a
> avg over all the workers) inside one worker.
>
OTOH if someone wants to parallelize scan (including expensive qual) and
On Sat, Feb 21, 2015 at 12:57 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Feb 18, 2015 at 6:44 PM, Andres Freund <andres@2ndquadrant.com> > wrote: >> On 2015-02-18 16:59:26 +0530, Amit Kapila wrote: >> >> > There could be some cases where it could be beneficial for worker >> > to process a sub-tree, but I think there will be more cases where >> > it will just work on a part of node and send the result back to either >> > master backend or another worker for further processing. >> >> I think many parallelism projects start out that way, and then notice >> that it doesn't parallelize very efficiently. >> >> The most extreme example, but common, is aggregation over large amounts >> of data - unless you want to ship huge amounts of data between processes >> eto parallize it you have to do the sequential scan and the >> pre-aggregate step (that e.g. selects count() and sum() to implement a >> avg over all the workers) inside one worker. >> > > OTOH if someone wants to parallelize scan (including expensive qual) and > sort then it will be better to perform scan (or part of scan by one worker) > and sort by other worker. There exists a performance problem if we perform SCAN in one worker and SORT operation in another worker, because there is a need of twice tuple transfer between worker to worker/backend. This is a costly operation. It is better to combine SCAN and SORT operation into a one worker job. This can be targeted once the parallel scan code is stable. Regards, Hari Babu Fujitsu Australia
On Tue, Feb 17, 2015 at 11:22 AM, Andres Freund <andres@2ndquadrant.com> wrote: >> I still think this belongs in heapam.c somehow or other. If the logic >> is all in the executor, then it becomes impossible for any code that >> doensn't use the executor to do a parallel heap scan, and that's >> probably bad. It's not hard to imagine something like CLUSTER wanting >> to reuse that code, and that won't be possible if the logic is up in >> some higher layer. > > Yea. > >> If the logic we want is to start with a large chunk size and then >> switch to a small chunk size when there's not much of the relation >> left to scan, there's still no reason that can't be encapsulated in >> heapam.c. > > I don't mind having some logic in there, but I think you put in too > much. The snapshot stuff should imo go, and the next page logic should > be caller provided. If we need to provide a way for the caller to provide the next-page logic, then I think that should be done via configuration arguments or flags, not a callback. There's just no way that the needs of the executor are going to be so radically different from a utility command that only a callback will do. >> I think it makes sense to think of a set of tasks in which workers can >> assist. So you a query tree which is just one query tree, with no >> copies of the nodes, and then there are certain places in that query >> tree where a worker can jump in and assist that node. To do that, it >> will have a copy of the node, but that doesn't mean that all of the >> stuff inside the node becomes shared data at the code level, because >> that would be stupid. > > My only "problem" with that description is that I think workers will > have to work on more than one node - it'll be entire subtrees of the > executor tree. Amit and I had a long discussion about this on Friday while in Boston together. I previously argued that the master and the slave should be executing the same node, ParallelSeqScan. However, Amit argued persuasively that what the master is doing is really pretty different from what the worker is doing, and that they really ought to be running two different nodes. This led us to cast about for a better design, and we came up with something that I think will be much better. The basic idea is to introduce a new node called Funnel. A Funnel node will have a left child but no right child, and its job will be to fire up a given number of workers. Each worker will execute the plan which is the left child of the funnel. The funnel node itself will pull tuples from all of those workers, and can also (if there are no tuples available from any worker) execute the plan itself. So a parallel sequential scan will look something like this: Funnel Workers: 4 -> Partial Heap Scan on xyz What this is saying is that each worker is going to scan part of the heap for xyz; together, they will scan the whole thing. The neat thing about this way of separating things out is that we can eventually write code to push more stuff down into the funnel. For example, consider this: Nested Loop -> Seq Scan on foo -> Index Scan on bar Index Cond: bar.x = foo.x Now, if a parallel sequential scan is cheaper than a regular sequential scan, we can instead do this: Nested Loop -> Funnel -> Partial Heap Scan on foo -> Index Scan on bara Index Cond: bar.x = foo.x The problem with this is that the nested loop/index scan is happening entirely in the master. But we can have logic that fixes that by knowing that a nested loop can be pushed through a funnel, yielding this: Funnel -> Nested Loop -> Partial Heap Scan on foo -> Index Scan on bar Index Cond: bar.x = foo.x Now that's pretty neat. One can also imagine doing this with aggregates. Consider: HashAggregate -> Funnel -> Partial Heap Scan on foo Filter: x = 1 Here, we can't just push the HashAggregate through the filter, but given infrastructure for we could convert that to something like this: HashAggregateFinish -> Funnel -> HashAggregatePartial -> Partial Heap Scan on foo Filter: x = 1 That'd be swell. You can see that something like this will also work for breaking off an entire plan tree and shoving it down into a worker. The master can't participate in the computation in that case, but it's otherwise the same idea. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> Amit and I had a long discussion about this on Friday while in Boston > together. I previously argued that the master and the slave should be > executing the same node, ParallelSeqScan. However, Amit argued > persuasively that what the master is doing is really pretty different > from what the worker is doing, and that they really ought to be > running two different nodes. This led us to cast about for a better > design, and we came up with something that I think will be much > better. > > The basic idea is to introduce a new node called Funnel. A Funnel > node will have a left child but no right child, and its job will be to > fire up a given number of workers. Each worker will execute the plan > which is the left child of the funnel. The funnel node itself will > pull tuples from all of those workers, and can also (if there are no > tuples available from any worker) execute the plan itself. So a > parallel sequential scan will look something like this: > > Funnel > Workers: 4 > -> Partial Heap Scan on xyz > > What this is saying is that each worker is going to scan part of the > heap for xyz; together, they will scan the whole thing. > What is the best way to determine the number of workers? Fixed number is an idea. It may also make sense to add a new common field to Path node to introduce how much portion of the node execution can be parallelized, or unavailable to run in parallel. Not on the plan time, we may be able to determine the number according to the number of concurrent workers and number of CPU cores. > The neat thing about this way of separating things out is that we can > eventually write code to push more stuff down into the funnel. For > example, consider this: > > Nested Loop > -> Seq Scan on foo > -> Index Scan on bar > Index Cond: bar.x = foo.x > > Now, if a parallel sequential scan is cheaper than a regular > sequential scan, we can instead do this: > > Nested Loop > -> Funnel > -> Partial Heap Scan on foo > -> Index Scan on bara > Index Cond: bar.x = foo.x > > The problem with this is that the nested loop/index scan is happening > entirely in the master. But we can have logic that fixes that by > knowing that a nested loop can be pushed through a funnel, yielding > this: > > Funnel > -> Nested Loop > -> Partial Heap Scan on foo > -> Index Scan on bar > Index Cond: bar.x = foo.x > > Now that's pretty neat. One can also imagine doing this with > aggregates. Consider: > I guess the planner enhancement shall exist around add_paths_to_joinrel(). In case when any underlying join paths that support multi-node execution, the new portion will add Funnel node with these join paths. Just my thought. > HashAggregate > -> Funnel > -> Partial Heap Scan on foo > Filter: x = 1 > > Here, we can't just push the HashAggregate through the filter, but > given infrastructure for we could convert that to something like this: > > HashAggregateFinish > -> Funnel > -> HashAggregatePartial > -> Partial Heap Scan on foo > Filter: x = 1 > > That'd be swell. > > You can see that something like this will also work for breaking off > an entire plan tree and shoving it down into a worker. The master > can't participate in the computation in that case, but it's otherwise > the same idea. > I believe the entire vision we've discussed around combining aggregate function thread is above, although people primarily considers to apply this feature on aggregate push-down across join. One key infrastructure may be a capability to define the combining function of aggregates. It informs the planner given aggregation support two stage execution. In addition to this, we may need to have a planner enhancement to inject the partial aggregate node during path construction. Probably, we have to set a flag to inform later stage (that will construct Agg plan) the underlying scan/join node takes partial aggregation, thus, final aggregation has to expect state data, instead of usual arguments for row-by-row. Also, I think HashJoin with very large outer relation but unbalanced much small inner is a good candidate to distribute multiple nodes. Even if multi-node HashJoin has to read the small inner relation N-times, separation of very large outer relation will make gain. Thanks, -- KaiGai Kohei <kaigai@kaigai.gr.jp>
>
> On Tue, Feb 17, 2015 at 11:22 AM, Andres Freund <andres@2ndquadrant.com> wrote:
> > My only "problem" with that description is that I think workers will
> > have to work on more than one node - it'll be entire subtrees of the
> > executor tree.
>
> Amit and I had a long discussion about this on Friday while in Boston
> together. I previously argued that the master and the slave should be
> executing the same node, ParallelSeqScan. However, Amit argued
> persuasively that what the master is doing is really pretty different
> from what the worker is doing, and that they really ought to be
> running two different nodes. This led us to cast about for a better
> design, and we came up with something that I think will be much
> better.
>
> The basic idea is to introduce a new node called Funnel. A Funnel
> node will have a left child but no right child, and its job will be to
> fire up a given number of workers. Each worker will execute the plan
> which is the left child of the funnel. The funnel node itself will
> pull tuples from all of those workers, and can also (if there are no
> tuples available from any worker) execute the plan itself.
Attachment
>
> On Sun, Feb 22, 2015 at 6:39 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > On Tue, Feb 17, 2015 at 11:22 AM, Andres Freund <andres@2ndquadrant.com> wrote:
> > > My only "problem" with that description is that I think workers will
> > > have to work on more than one node - it'll be entire subtrees of the
> > > executor tree.
> >
> > Amit and I had a long discussion about this on Friday while in Boston
> > together. I previously argued that the master and the slave should be
> > executing the same node, ParallelSeqScan. However, Amit argued
> > persuasively that what the master is doing is really pretty different
> > from what the worker is doing, and that they really ought to be
> > running two different nodes. This led us to cast about for a better
> > design, and we came up with something that I think will be much
> > better.
> >
> > The basic idea is to introduce a new node called Funnel. A Funnel
> > node will have a left child but no right child, and its job will be to
> > fire up a given number of workers. Each worker will execute the plan
> > which is the left child of the funnel. The funnel node itself will
> > pull tuples from all of those workers, and can also (if there are no
> > tuples available from any worker) execute the plan itself.
>
> I have modified the patch to introduce a Funnel node (and left child
> as PartialSeqScan node). Apart from that, some other noticeable
> changes based on feedback include:
> a) Master backend forms and send the planned stmt to each worker,
> earlier patch use to send individual elements and form the planned
> stmt in each worker.
> b) Passed tuples directly via tuple queue instead of going via
> FE-BE protocol.
> c) Removed restriction of expressions in target list.
> d) Introduced a parallelmodeneeded flag in plannerglobal structure
> and set it for Funnel plan.
>
> There is still some work left like integrating with
> access-parallel-safety patch (use parallelmodeok flag to decide
> whether parallel path can be generated, Enter/Exit parallel mode is still
> done during execution of funnel node).
>
> I think these are minor points which can be fixed once we decide
> on the other major parts of patch. Find modified patch attached with
> this mail.
>
> Note -
> This patch is based on Head (commit-id: d1479011) +
> parallel-mode-v6.patch [1] + parallel-heap-scan.patch[2]
>
> [1]
> http://www.postgresql.org/message-id/CA+TgmobCMwFOz-9=hFv=hJ4SH7p=5X6Ga5V=WtT8=huzE6C+Mg@mail.gmail.com
> [2]
> http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
>
Attachment
On Tue, Mar 10, 2015 at 1:38 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > > Assuming previous patch is in right direction, I have enabled > join support for the patch and done some minor cleanup of > patch which leads to attached new version. Is this patch handles the cases where the re-scan starts without finishing the earlier scan? Regards, Hari Babu Fujitsu Australia
>
> On Tue, Mar 10, 2015 at 1:38 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > Assuming previous patch is in right direction, I have enabled
> > join support for the patch and done some minor cleanup of
> > patch which leads to attached new version.
>
> Is this patch handles the cases where the re-scan starts without
> finishing the earlier scan?
>
Do you mean to say cases like ANTI, SEMI Join (in nodeNestLoop.c)
On Tue, Mar 10, 2015 at 3:09 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Tue, Mar 10, 2015 at 6:50 AM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> >> On Tue, Mar 10, 2015 at 1:38 AM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > >> > Assuming previous patch is in right direction, I have enabled >> > join support for the patch and done some minor cleanup of >> > patch which leads to attached new version. >> >> Is this patch handles the cases where the re-scan starts without >> finishing the earlier scan? >> > > Do you mean to say cases like ANTI, SEMI Join (in nodeNestLoop.c) > where we scan the next outer tuple and rescan inner table without > completing the previous scan of inner table? Yes. > I have currently modelled it based on existing rescan for seqscan > (ExecReScanSeqScan()) which means it will begin the scan again. > Basically if the workers are already started/initialized by previous > scan, then re-initialize them (refer function ExecReScanFunnel() in > patch). > > Can you elaborate more if you think current handling is not sufficient > for any case? From ExecReScanFunnel function it seems that the re-scan waits till all the workers has to be finished to start again the next scan. Are the workers will stop the current ongoing task? otherwise this may decrease the performance instead of improving as i feel. I am not sure if it already handled or not, when a worker is waiting to pass the results, whereas the backend is trying to start the re-scan? Regards, Hari Babu Fujitsu Australia
>
> On Tue, Mar 10, 2015 at 3:09 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> > I have currently modelled it based on existing rescan for seqscan
> > (ExecReScanSeqScan()) which means it will begin the scan again.
> > Basically if the workers are already started/initialized by previous
> > scan, then re-initialize them (refer function ExecReScanFunnel() in
> > patch).
> >
> > Can you elaborate more if you think current handling is not sufficient
> > for any case?
>
> From ExecReScanFunnel function it seems that the re-scan waits till
> all the workers
> has to be finished to start again the next scan. Are the workers will
> stop the current
> ongoing task? otherwise this may decrease the performance instead of
> improving as i feel.
>
as DestroyParallelContext() will automatically terminate all the workers.
> to pass the results,
> whereas the backend is trying to start the re-scan?
>
On Tue, Mar 3, 2015 at 7:47 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I have modified the patch to introduce a Funnel node (and left child > as PartialSeqScan node). Apart from that, some other noticeable > changes based on feedback include: > a) Master backend forms and send the planned stmt to each worker, > earlier patch use to send individual elements and form the planned > stmt in each worker. > b) Passed tuples directly via tuple queue instead of going via > FE-BE protocol. > c) Removed restriction of expressions in target list. > d) Introduced a parallelmodeneeded flag in plannerglobal structure > and set it for Funnel plan. > > There is still some work left like integrating with > access-parallel-safety patch (use parallelmodeok flag to decide > whether parallel path can be generated, Enter/Exit parallel mode is still > done during execution of funnel node). > > I think these are minor points which can be fixed once we decide > on the other major parts of patch. Find modified patch attached with > this mail. This is definitely progress. I do think you need to integrate it with the access-parallel-safety patch. Other comments: - There's not much code left in shmmqam.c. I think that the remaining logic should be integrated directly into nodeFunnel.c, with the two bools in worker_result_state becoming part of the FunnelState. It doesn't make sense to have a separate structure for two booleans and 20 lines of code. If you were going to keep this file around, I'd complain about its name and its location in the source tree, too, but as it is I think we can just get rid of it altogether. - Something is deeply wrong with the separation of concerns between nodeFunnel.c and nodePartialSeqscan.c. nodeFunnel.c should work correctly with *any arbitrary plan tree* as its left child, and that is clearly not the case right now. shm_getnext() can't just do heap_getnext(). Instead, it's got to call ExecProcNode() on its left child and let the left child decide what to do about that. The logic in InitFunnelRelation() belongs in the parallel seq scan node, not the funnel. ExecReScanFunnel() cannot be calling heap_parallel_rescan(); it needs to *not know* that there is a parallel scan under it. The comment in FunnelRecheck is a copy-and-paste from elsewhere that is not applicable to a generic funnel mode. - The comment in execAmi.c refers to says "Backward scan is not suppotted for parallel sequiantel scan". "Sequential" is mis-spelled here, but I think you should just nuke the whole comment. The funnel node is not, in the long run, just for parallel sequential scan, so putting that comment above it is not right. If you want to keep the comment, it's got to be more general than that somehow, like "parallel nodes do not support backward scans", but I'd just drop it. - Can we rename create_worker_scan_plannedstmt to create_parallel_worker_plannedstmt? - I *strongly* suggest that, for the first version of this, we remove all of the tts_fromheap stuff. Let's make no special provision for returning a tuple stored in a tuple queue; instead, just copy it and store it in the slot as a pfree-able tuple. That may be slightly less efficient, but I think it's totally worth it to avoid the complexity of tinkering with the slot mechanism. - InstrAggNode claims that we only need the master's information for statistics other than buffer usage and tuple counts, but is that really true? The parallel backends can be working on the parallel part of the plan while the master is doing something else, so the amount of time the *master* spent in a particular node may not be that relevant. We might need to think carefully about what it makes sense to display in the EXPLAIN output in parallel cases. - The header comment on nodeFunnel.h capitalizes the filename incorrectly. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Mar 11, 2015 at 6:31 AM, Robert Haas <robertmhaas@gmail.com> wrote: > On Tue, Mar 3, 2015 at 7:47 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I have modified the patch to introduce a Funnel node (and left child >> as PartialSeqScan node). Apart from that, some other noticeable >> changes based on feedback include: >> a) Master backend forms and send the planned stmt to each worker, >> earlier patch use to send individual elements and form the planned >> stmt in each worker. >> b) Passed tuples directly via tuple queue instead of going via >> FE-BE protocol. >> c) Removed restriction of expressions in target list. >> d) Introduced a parallelmodeneeded flag in plannerglobal structure >> and set it for Funnel plan. >> >> There is still some work left like integrating with >> access-parallel-safety patch (use parallelmodeok flag to decide >> whether parallel path can be generated, Enter/Exit parallel mode is still >> done during execution of funnel node). >> >> I think these are minor points which can be fixed once we decide >> on the other major parts of patch. Find modified patch attached with >> this mail. > > - Something is deeply wrong with the separation of concerns between > nodeFunnel.c and nodePartialSeqscan.c. nodeFunnel.c should work > correctly with *any arbitrary plan tree* as its left child, and that > is clearly not the case right now. shm_getnext() can't just do > heap_getnext(). Instead, it's got to call ExecProcNode() on its left > child and let the left child decide what to do about that. The logic > in InitFunnelRelation() belongs in the parallel seq scan node, not the > funnel. ExecReScanFunnel() cannot be calling heap_parallel_rescan(); > it needs to *not know* that there is a parallel scan under it. The > comment in FunnelRecheck is a copy-and-paste from elsewhere that is > not applicable to a generic funnel mode. In create_parallelscan_paths() function the funnel path is added once the partial seq scan path is generated. I feel the funnel path can be added once on top of the total possible parallel path in the entire query path. Is this the right patch to add such support also? Regards, Hari Babu Fujitsu Australia
On 10-03-2015 PM 01:09, Amit Kapila wrote: > On Tue, Mar 10, 2015 at 6:50 AM, Haribabu Kommi <kommi.haribabu@gmail.com> >> Is this patch handles the cases where the re-scan starts without >> finishing the earlier scan? >> > > Do you mean to say cases like ANTI, SEMI Join (in nodeNestLoop.c) > where we scan the next outer tuple and rescan inner table without > completing the previous scan of inner table? > > I have currently modelled it based on existing rescan for seqscan > (ExecReScanSeqScan()) which means it will begin the scan again. > Basically if the workers are already started/initialized by previous > scan, then re-initialize them (refer function ExecReScanFunnel() in > patch). > From Robert's description[1], it looked like the NestLoop with Funnel would have Funnel as either outer plan or topmost plan node or NOT a parameterised plan. In that case, would this case arise or am I missing something? Thanks, Amit [1] http://www.postgresql.org/message-id/CA+TgmobM7X6jgre442638b+33h1EWa=vcZqnsvzEdX057ZHVuw@mail.gmail.com
>
> On Tue, Mar 3, 2015 at 7:47 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > There is still some work left like integrating with
> > access-parallel-safety patch (use parallelmodeok flag to decide
> > whether parallel path can be generated, Enter/Exit parallel mode is still
> > done during execution of funnel node).
> >
> > I think these are minor points which can be fixed once we decide
> > on the other major parts of patch. Find modified patch attached with
> > this mail.
>
> This is definitely progress. I do think you need to integrate it with
> the access-parallel-safety patch.
>
> - There's not much code left in shmmqam.c. I think that the remaining
> logic should be integrated directly into nodeFunnel.c, with the two
> bools in worker_result_state becoming part of the FunnelState. It
> doesn't make sense to have a separate structure for two booleans and
> 20 lines of code. If you were going to keep this file around, I'd
> complain about its name and its location in the source tree, too, but
> as it is I think we can just get rid of it altogether.
>
> - Something is deeply wrong with the separation of concerns between
> nodeFunnel.c and nodePartialSeqscan.c. nodeFunnel.c should work
> correctly with *any arbitrary plan tree* as its left child, and that
> is clearly not the case right now. shm_getnext() can't just do
> heap_getnext(). Instead, it's got to call ExecProcNode() on its left
> child and let the left child decide what to do about that.
> in InitFunnelRelation() belongs in the parallel seq scan node, not the
> funnel.
> it needs to *not know* that there is a parallel scan under it.
> comment in FunnelRecheck is a copy-and-paste from elsewhere that is
> not applicable to a generic funnel mode.
>
> - The comment in execAmi.c refers to says "Backward scan is not
> suppotted for parallel sequiantel scan". "Sequential" is mis-spelled
> here, but I think you should just nuke the whole comment. The funnel
> node is not, in the long run, just for parallel sequential scan, so
> putting that comment above it is not right. If you want to keep the
> comment, it's got to be more general than that somehow, like "parallel
> nodes do not support backward scans", but I'd just drop it.
>
> - Can we rename create_worker_scan_plannedstmt to
> create_parallel_worker_plannedstmt?
>
> - I *strongly* suggest that, for the first version of this, we remove
> all of the tts_fromheap stuff. Let's make no special provision for
> returning a tuple stored in a tuple queue; instead, just copy it and
> store it in the slot as a pfree-able tuple. That may be slightly less
> efficient, but I think it's totally worth it to avoid the complexity
> of tinkering with the slot mechanism.
>
> - InstrAggNode claims that we only need the master's information for
> statistics other than buffer usage and tuple counts, but is that
> really true? The parallel backends can be working on the parallel
> part of the plan while the master is doing something else, so the
> amount of time the *master* spent in a particular node may not be that
> relevant.
> to display in the EXPLAIN output in parallel cases.
>
>
Attachment
On 12 March 2015 at 14:46, Amit Kapila <amit.kapila16@gmail.com> wrote: > One additional change (we need to SetLatch() in > HandleParallelMessageInterrupt) > is done to handle the hang issue reported on parallel-mode thread. > Without this change it is difficult to verify the patch (will remove this > change > once new version of parallel-mode patch containing this change will be > posted). Applied parallel-mode-v7.patch and parallel_seqscan_v10.patch, but getting this error when building: gcc -Wall -Wmissing-prototypes -Wpointer-arith -Wdeclaration-after-statement -Wendif-labels -Wmissing-format-attribute -Wformat-security -fno-strict-aliasing -fwrapv -fexcess-precision=standard -O2 -I../../../../src/include -D_GNU_SOURCE -c -o brin.o brin.c -MMD -MP -MF .deps/brin.Po In file included from ../../../../src/include/nodes/execnodes.h:18:0, from ../../../../src/include/access/brin.h:14, from brin.c:18: ../../../../src/include/access/heapam.h:119:34: error: unknown type name ‘ParallelHeapScanDesc’extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan); ^ Am I missing another patch here? -- Thom
>
> On 12 March 2015 at 14:46, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > One additional change (we need to SetLatch() in
> > HandleParallelMessageInterrupt)
> > is done to handle the hang issue reported on parallel-mode thread.
> > Without this change it is difficult to verify the patch (will remove this
> > change
> > once new version of parallel-mode patch containing this change will be
> > posted).
>
> Applied parallel-mode-v7.patch and parallel_seqscan_v10.patch, but
> getting this error when building:
>
> gcc -Wall -Wmissing-prototypes -Wpointer-arith
> -Wdeclaration-after-statement -Wendif-labels
> -Wmissing-format-attribute -Wformat-security -fno-strict-aliasing
> -fwrapv -fexcess-precision=standard -O2 -I../../../../src/include
> -D_GNU_SOURCE -c -o brin.o brin.c -MMD -MP -MF .deps/brin.Po
> In file included from ../../../../src/include/nodes/execnodes.h:18:0,
> from ../../../../src/include/access/brin.h:14,
> from brin.c:18:
> ../../../../src/include/access/heapam.h:119:34: error: unknown type
> name ‘ParallelHeapScanDesc’
> extern void heap_parallel_rescan(ParallelHeapScanDesc pscan,
> HeapScanDesc scan);
> ^
>
> Am I missing another patch here?
http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
On 12 March 2015 at 15:29, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Mar 12, 2015 at 8:33 PM, Thom Brown <thom@linux.com> wrote: >> >> On 12 March 2015 at 14:46, Amit Kapila <amit.kapila16@gmail.com> wrote: >> > One additional change (we need to SetLatch() in >> > HandleParallelMessageInterrupt) >> > is done to handle the hang issue reported on parallel-mode thread. >> > Without this change it is difficult to verify the patch (will remove >> > this >> > change >> > once new version of parallel-mode patch containing this change will be >> > posted). >> >> Applied parallel-mode-v7.patch and parallel_seqscan_v10.patch, but >> getting this error when building: >> >> gcc -Wall -Wmissing-prototypes -Wpointer-arith >> -Wdeclaration-after-statement -Wendif-labels >> -Wmissing-format-attribute -Wformat-security -fno-strict-aliasing >> -fwrapv -fexcess-precision=standard -O2 -I../../../../src/include >> -D_GNU_SOURCE -c -o brin.o brin.c -MMD -MP -MF .deps/brin.Po >> In file included from ../../../../src/include/nodes/execnodes.h:18:0, >> from ../../../../src/include/access/brin.h:14, >> from brin.c:18: >> ../../../../src/include/access/heapam.h:119:34: error: unknown type >> name ‘ParallelHeapScanDesc’ >> extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, >> HeapScanDesc scan); >> ^ >> >> Am I missing another patch here? > > Yes, the below parallel-heap-scan patch. > http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com > > Please note that parallel_setup_cost and parallel_startup_cost are > still set to zero by default, so you need to set it to higher values > if you don't want the parallel plans once parallel_seqscan_degree > is set. I have yet to comeup with default values for them, needs > some tests. Thanks. Getting a problem: createdb pgbench pgbench -i -s 200 pgbench CREATE TABLE pgbench_accounts_1 (CHECK (bid = 1)) INHERITS (pgbench_accounts); ... CREATE TABLE pgbench_accounts_200 (CHECK (bid = 200)) INHERITS (pgbench_accounts); WITH del AS (DELETE FROM pgbench_accounts WHERE bid = 1 RETURNING *) INSERT INTO pgbench_accounts_1 SELECT * FROM del; ... WITH del AS (DELETE FROM pgbench_accounts WHERE bid = 200 RETURNING *) INSERT INTO pgbench_accounts_200 SELECT * FROM del; VACUUM ANALYSE; # SELECT name, setting FROM pg_settings WHERE name IN ('parallel_seqscan_degree','max_worker_processes','seq_page_cost'); name | setting -------------------------+---------max_worker_processes | 20parallel_seqscan_degree | 8seq_page_cost | 1000 (3 rows) # EXPLAIN SELECT DISTINCT bid FROM pgbench_accounts; ERROR: too many dynamic shared memory segments And separately, I've seen this in the logs: 2015-03-12 16:09:30 GMT [7880]: [4-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [5-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [6-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [7-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [8-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [9-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [10-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [11-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [12-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [13-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [14-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [15-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [16-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [17-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [18-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [19-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [20-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7913) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [21-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [22-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7919) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [23-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [24-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7916) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [25-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [26-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7918) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [27-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [28-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7917) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [29-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [30-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7914) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [31-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [32-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7915) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [33-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [34-1] user=,db=,client= LOG: worker process: parallel worker for PID 7889 (PID 7912) exited with exit code 0 2015-03-12 16:09:30 GMT [7880]: [35-1] user=,db=,client= LOG: unregistering background worker "parallel worker for PID 7889" 2015-03-12 16:09:30 GMT [7880]: [36-1] user=,db=,client= LOG: server process (PID 7889) was terminated by signal 11: Segmentation fault 2015-03-12 16:09:30 GMT [7880]: [37-1] user=,db=,client= DETAIL: Failed process was running: SELECT pg_catalog.quote_ident(c.relname) FROM pg_catalog.pg_class c WHERE c.relkind IN ('r', 'S', 'v', 'm', 'f') AND substring(pg_catalog.quote_ident(c.relname),1,10)='pgbench_br' AND pg_catalog.pg_table_is_visible(c.oid) AND c.relnamespace <> (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname = 'pg_catalog') UNION SELECT pg_catalog.quote_ident(n.nspname)|| '.' FROM pg_catalog.pg_namespace n WHERE substring(pg_catalog.quote_ident(n.nspname) || '.',1,10)='pgbench_br' AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,10) = substring('pgbench_br',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) > 1 UNION SELECT pg_catalog.quote_ident(n.nspname) || '.' || pg_catalog.quote_ident(c.relname) FROM pg_catalog.pg_class c, pg_catalog.pg_namespace n WHERE c.relnamespace = n.oid AND c.relkind IN ('r', 'S', 'v', 'm', 'f') AND substring(pg_catalog.quote_ident(n.nspname) || '.' || pg_catalog.quote_ident(c.relname),1,10)='pgbench_br' AND substri 2015-03-12 16:09:30 GMT [7880]: [38-1] user=,db=,client= LOG: terminating any other active server processes 2015-03-12 16:09:30 GMT [7886]: [2-1] user=,db=,client= WARNING: terminating connection because of crash of another server process 2015-03-12 16:09:30 GMT [7886]: [3-1] user=,db=,client= DETAIL: The postmaster has commanded this server process to roll back the current transaction and exit, because another server process exited abnormally and possibly corrupted shared memory. 2015-03-12 16:09:30 GMT [7886]: [4-1] user=,db=,client= HINT: In a moment you should be able to reconnect to the database and repeat your command. 2015-03-12 16:09:30 GMT [7880]: [39-1] user=,db=,client= LOG: all server processes terminated; reinitializing 2015-03-12 16:09:30 GMT [7920]: [1-1] user=,db=,client= LOG: database system was interrupted; last known up at 2015-03-12 16:07:26 GMT 2015-03-12 16:09:30 GMT [7920]: [2-1] user=,db=,client= LOG: database system was not properly shut down; automatic recovery in progress 2015-03-12 16:09:30 GMT [7920]: [3-1] user=,db=,client= LOG: invalid record length at 2/7E269A0 2015-03-12 16:09:30 GMT [7920]: [4-1] user=,db=,client= LOG: redo is not required 2015-03-12 16:09:30 GMT [7880]: [40-1] user=,db=,client= LOG: database system is ready to accept connections 2015-03-12 16:09:30 GMT [7924]: [1-1] user=,db=,client= LOG: autovacuum launcher started I can recreate this by typing: EXPLAIN SELECT DISTINCT bid FROM pgbench_<tab> This happens with seq_page_cost = 1000, but not when it's set to 1. -- Thom
On 12 March 2015 at 16:20, Thom Brown <thom@linux.com> wrote: > On 12 March 2015 at 15:29, Amit Kapila <amit.kapila16@gmail.com> wrote: >> On Thu, Mar 12, 2015 at 8:33 PM, Thom Brown <thom@linux.com> wrote: >>> >>> On 12 March 2015 at 14:46, Amit Kapila <amit.kapila16@gmail.com> wrote: >>> > One additional change (we need to SetLatch() in >>> > HandleParallelMessageInterrupt) >>> > is done to handle the hang issue reported on parallel-mode thread. >>> > Without this change it is difficult to verify the patch (will remove >>> > this >>> > change >>> > once new version of parallel-mode patch containing this change will be >>> > posted). >>> >>> Applied parallel-mode-v7.patch and parallel_seqscan_v10.patch, but >>> getting this error when building: >>> >>> gcc -Wall -Wmissing-prototypes -Wpointer-arith >>> -Wdeclaration-after-statement -Wendif-labels >>> -Wmissing-format-attribute -Wformat-security -fno-strict-aliasing >>> -fwrapv -fexcess-precision=standard -O2 -I../../../../src/include >>> -D_GNU_SOURCE -c -o brin.o brin.c -MMD -MP -MF .deps/brin.Po >>> In file included from ../../../../src/include/nodes/execnodes.h:18:0, >>> from ../../../../src/include/access/brin.h:14, >>> from brin.c:18: >>> ../../../../src/include/access/heapam.h:119:34: error: unknown type >>> name ‘ParallelHeapScanDesc’ >>> extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, >>> HeapScanDesc scan); >>> ^ >>> >>> Am I missing another patch here? >> >> Yes, the below parallel-heap-scan patch. >> http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com >> >> Please note that parallel_setup_cost and parallel_startup_cost are >> still set to zero by default, so you need to set it to higher values >> if you don't want the parallel plans once parallel_seqscan_degree >> is set. I have yet to comeup with default values for them, needs >> some tests. > > Thanks. Getting a problem: > > createdb pgbench > pgbench -i -s 200 pgbench > > CREATE TABLE pgbench_accounts_1 (CHECK (bid = 1)) INHERITS (pgbench_accounts); > ... > CREATE TABLE pgbench_accounts_200 (CHECK (bid = 200)) INHERITS > (pgbench_accounts); > > WITH del AS (DELETE FROM pgbench_accounts WHERE bid = 1 RETURNING *) > INSERT INTO pgbench_accounts_1 SELECT * FROM del; > ... > WITH del AS (DELETE FROM pgbench_accounts WHERE bid = 200 RETURNING *) > INSERT INTO pgbench_accounts_200 SELECT * FROM del; > > VACUUM ANALYSE; > > # SELECT name, setting FROM pg_settings WHERE name IN > ('parallel_seqscan_degree','max_worker_processes','seq_page_cost'); > name | setting > -------------------------+--------- > max_worker_processes | 20 > parallel_seqscan_degree | 8 > seq_page_cost | 1000 > (3 rows) > > # EXPLAIN SELECT DISTINCT bid FROM pgbench_accounts; > ERROR: too many dynamic shared memory segments > > > And separately, I've seen this in the logs: > > 2015-03-12 16:09:30 GMT [7880]: [4-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [5-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [6-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [7-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [8-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [9-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [10-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [11-1] user=,db=,client= LOG: > registering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [12-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [13-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [14-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [15-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [16-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [17-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [18-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [19-1] user=,db=,client= LOG: > starting background worker process "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [20-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7913) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [21-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [22-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7919) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [23-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [24-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7916) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [25-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [26-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7918) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [27-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [28-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7917) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [29-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [30-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7914) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [31-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [32-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7915) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [33-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [34-1] user=,db=,client= LOG: worker > process: parallel worker for PID 7889 (PID 7912) exited with exit code > 0 > 2015-03-12 16:09:30 GMT [7880]: [35-1] user=,db=,client= LOG: > unregistering background worker "parallel worker for PID 7889" > 2015-03-12 16:09:30 GMT [7880]: [36-1] user=,db=,client= LOG: server > process (PID 7889) was terminated by signal 11: Segmentation fault > 2015-03-12 16:09:30 GMT [7880]: [37-1] user=,db=,client= DETAIL: > Failed process was running: SELECT pg_catalog.quote_ident(c.relname) > FROM pg_catalog.pg_class c WHERE c.relkind IN ('r', 'S', 'v', 'm', > 'f') AND substring(pg_catalog.quote_ident(c.relname),1,10)='pgbench_br' > AND pg_catalog.pg_table_is_visible(c.oid) AND c.relnamespace <> > (SELECT oid FROM pg_catalog.pg_namespace WHERE nspname = 'pg_catalog') > UNION > SELECT pg_catalog.quote_ident(n.nspname) || '.' FROM > pg_catalog.pg_namespace n WHERE > substring(pg_catalog.quote_ident(n.nspname) || '.',1,10)='pgbench_br' > AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE > substring(pg_catalog.quote_ident(nspname) || '.',1,10) = > substring('pgbench_br',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) >> 1 > UNION > SELECT pg_catalog.quote_ident(n.nspname) || '.' || > pg_catalog.quote_ident(c.relname) FROM pg_catalog.pg_class c, > pg_catalog.pg_namespace n WHERE c.relnamespace = n.oid AND c.relkind > IN ('r', 'S', 'v', 'm', 'f') AND > substring(pg_catalog.quote_ident(n.nspname) || '.' || > pg_catalog.quote_ident(c.relname),1,10)='pgbench_br' AND substri > 2015-03-12 16:09:30 GMT [7880]: [38-1] user=,db=,client= LOG: > terminating any other active server processes > 2015-03-12 16:09:30 GMT [7886]: [2-1] user=,db=,client= WARNING: > terminating connection because of crash of another server process > 2015-03-12 16:09:30 GMT [7886]: [3-1] user=,db=,client= DETAIL: The > postmaster has commanded this server process to roll back the current > transaction and exit, because another server process exited abnormally > and possibly corrupted shared memory. > 2015-03-12 16:09:30 GMT [7886]: [4-1] user=,db=,client= HINT: In a > moment you should be able to reconnect to the database and repeat your > command. > 2015-03-12 16:09:30 GMT [7880]: [39-1] user=,db=,client= LOG: all > server processes terminated; reinitializing > 2015-03-12 16:09:30 GMT [7920]: [1-1] user=,db=,client= LOG: database > system was interrupted; last known up at 2015-03-12 16:07:26 GMT > 2015-03-12 16:09:30 GMT [7920]: [2-1] user=,db=,client= LOG: database > system was not properly shut down; automatic recovery in progress > 2015-03-12 16:09:30 GMT [7920]: [3-1] user=,db=,client= LOG: invalid > record length at 2/7E269A0 > 2015-03-12 16:09:30 GMT [7920]: [4-1] user=,db=,client= LOG: redo is > not required > 2015-03-12 16:09:30 GMT [7880]: [40-1] user=,db=,client= LOG: > database system is ready to accept connections > 2015-03-12 16:09:30 GMT [7924]: [1-1] user=,db=,client= LOG: > autovacuum launcher started > > I can recreate this by typing: > > EXPLAIN SELECT DISTINCT bid FROM pgbench_<tab> > > This happens with seq_page_cost = 1000, but not when it's set to 1. Another problem. I restarted the instance (just in case), and get this error: # \df+ *.* ERROR: cannot retain locks acquired while in parallel mode I get this even with seq_page_cost = 1, parallel_seqscan_degree = 1 and max_worker_processes = 1. -- Thom
>
> On 10-03-2015 PM 01:09, Amit Kapila wrote:
> > On Tue, Mar 10, 2015 at 6:50 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
> >> Is this patch handles the cases where the re-scan starts without
> >> finishing the earlier scan?
> >>
> >
> > Do you mean to say cases like ANTI, SEMI Join (in nodeNestLoop.c)
> > where we scan the next outer tuple and rescan inner table without
> > completing the previous scan of inner table?
> >
> > I have currently modelled it based on existing rescan for seqscan
> > (ExecReScanSeqScan()) which means it will begin the scan again.
> > Basically if the workers are already started/initialized by previous
> > scan, then re-initialize them (refer function ExecReScanFunnel() in
> > patch).
> >
>
> From Robert's description[1], it looked like the NestLoop with Funnel would
> have Funnel as either outer plan or topmost plan node or NOT a parameterised
> plan. In that case, would this case arise or am I missing something?
>
Probably not if the costing is right and user doesn't manually disable
On 13-03-2015 AM 10:24, Amit Kapila wrote: > On Thu, Mar 12, 2015 at 4:22 PM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> >> From Robert's description[1], it looked like the NestLoop with Funnel > would >> have Funnel as either outer plan or topmost plan node or NOT a > parameterised >> plan. In that case, would this case arise or am I missing something? >> > > Probably not if the costing is right and user doesn't manually disable > plans (like by set enable_* = off). However we should have rescan code > incase it chooses the plan such that Funnel is inner node and I think > apart from that also in few cases Rescan is required. > I see, thanks. By the way, is it right that TupleQueueFunnel.queue has one shm_mq_handle per initialized parallel worker? If so, how does TupleQueueFunnel.maxqueues relate to ParallelContext.nworkers (of the corresponding parallel context)? Why I asked this is because in CreateTupleQueueFunnel(): funnel->maxqueues = 8; funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *)); So, is the hardcoded "8" intentional or an oversight? Thanks, Amit
On 13-03-2015 PM 01:37, Amit Langote wrote: > By the way, is it right that TupleQueueFunnel.queue has one shm_mq_handle per > initialized parallel worker? If so, how does TupleQueueFunnel.maxqueues relate > to ParallelContext.nworkers (of the corresponding parallel context)? > > Why I asked this is because in CreateTupleQueueFunnel(): > > funnel->maxqueues = 8; > funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *)); > > So, is the hardcoded "8" intentional or an oversight? > Oh, I see that in RegisterTupleQueueOnFunnel(), the TupleQueueFunnel.queue is expanded (repalloc'd) if needed as per corresponding pcxt->nworkers. Thanks, Amit
On 12-03-2015 PM 11:46, Amit Kapila wrote: > [parallel_seqscan_v10.patch] There may be a bug in TupleQueueFunnelNext(). 1) I observed a hang with stack looking like: #0 0x00000039696df098 in poll () from /lib64/libc.so.6 #1 0x00000000006f1c6a in WaitLatchOrSocket (latch=0x7f29dc3c73b4, wakeEvents=1, sock=-1, timeout=0) at pg_latch.c:333 #2 0x00000000006f1aca in WaitLatch (latch=0x7f29dc3c73b4, wakeEvents=1, timeout=0) at pg_latch.c:197 #3 0x000000000065088b in TupleQueueFunnelNext (funnel=0x17b4a20, nowait=0 '\000', done=0x17ad481 "") at tqueue.c:269 #4 0x0000000000636cab in funnel_getnext (funnelstate=0x17ad3d0) at nodeFunnel.c:347 ... <snip> 2) In some cases, there can be a segmentation fault with stack looking like: #0 0x000000396968990a in memcpy () from /lib64/libc.so.6 #1 0x00000000006507e7 in TupleQueueFunnelNext (funnel=0x263c800, nowait=0 '\000', done=0x2633461 "") at tqueue.c:233 #2 0x0000000000636cab in funnel_getnext (funnelstate=0x26333b0) at nodeFunnel.c:347 #3 0x0000000000636901 in ExecFunnel (node=0x26333b0) at nodeFunnel.c:179 ... <snip> I could get rid of (1) and (2) with the attached fix.
On 13-03-2015 PM 05:32, Amit Langote wrote: > On 12-03-2015 PM 11:46, Amit Kapila wrote: >> [parallel_seqscan_v10.patch] > > There may be a bug in TupleQueueFunnelNext(). > > 1) I observed a hang with stack looking like: > > #0 0x00000039696df098 in poll () from /lib64/libc.so.6 > #1 0x00000000006f1c6a in WaitLatchOrSocket (latch=0x7f29dc3c73b4, > wakeEvents=1, sock=-1, timeout=0) at pg_latch.c:333 > #2 0x00000000006f1aca in WaitLatch (latch=0x7f29dc3c73b4, wakeEvents=1, > timeout=0) at pg_latch.c:197 > #3 0x000000000065088b in TupleQueueFunnelNext (funnel=0x17b4a20, nowait=0 > '\000', done=0x17ad481 "") at tqueue.c:269 > #4 0x0000000000636cab in funnel_getnext (funnelstate=0x17ad3d0) at > nodeFunnel.c:347 > ... > <snip> > > 2) In some cases, there can be a segmentation fault with stack looking like: > > #0 0x000000396968990a in memcpy () from /lib64/libc.so.6 > #1 0x00000000006507e7 in TupleQueueFunnelNext (funnel=0x263c800, nowait=0 > '\000', done=0x2633461 "") at tqueue.c:233 > #2 0x0000000000636cab in funnel_getnext (funnelstate=0x26333b0) at > nodeFunnel.c:347 > #3 0x0000000000636901 in ExecFunnel (node=0x26333b0) at nodeFunnel.c:179 > ... > <snip> > > I could get rid of (1) and (2) with the attached fix. Hit send too soon! By the way, the bug seems to be exposed only with a certain pattern/sequence of workers being detached (perhaps in immediate successive) whereby the funnel->nextqueue remains incorrectly set. The patch attached this time. By the way, when I have asserts enabled, I hit this compilation error: createplan.c: In function ‘create_partialseqscan_plan’: createplan.c:1180: error: ‘Path’ has no member named ‘path’ I see following line there: Assert(best_path->path.parent->rtekind == RTE_RELATION); Thanks, Amit
Attachment
>
> On 13-03-2015 PM 05:32, Amit Langote wrote:
> > On 12-03-2015 PM 11:46, Amit Kapila wrote:
> >> [parallel_seqscan_v10.patch]
> >
> > There may be a bug in TupleQueueFunnelNext().
> >
> > 1) I observed a hang with stack looking like:
> >
> > #0 0x00000039696df098 in poll () from /lib64/libc.so.6
> > #1 0x00000000006f1c6a in WaitLatchOrSocket (latch=0x7f29dc3c73b4,
> > wakeEvents=1, sock=-1, timeout=0) at pg_latch.c:333
> > #2 0x00000000006f1aca in WaitLatch (latch=0x7f29dc3c73b4, wakeEvents=1,
> > timeout=0) at pg_latch.c:197
> > #3 0x000000000065088b in TupleQueueFunnelNext (funnel=0x17b4a20, nowait=0
> > '\000', done=0x17ad481 "") at tqueue.c:269
> > #4 0x0000000000636cab in funnel_getnext (funnelstate=0x17ad3d0) at
> > nodeFunnel.c:347
> > ...
> > <snip>
> >
> > 2) In some cases, there can be a segmentation fault with stack looking like:
> >
> > #0 0x000000396968990a in memcpy () from /lib64/libc.so.6
> > #1 0x00000000006507e7 in TupleQueueFunnelNext (funnel=0x263c800, nowait=0
> > '\000', done=0x2633461 "") at tqueue.c:233
> > #2 0x0000000000636cab in funnel_getnext (funnelstate=0x26333b0) at
> > nodeFunnel.c:347
> > #3 0x0000000000636901 in ExecFunnel (node=0x26333b0) at nodeFunnel.c:179
> > ...
> > <snip>
> >
> > I could get rid of (1) and (2) with the attached fix.
>
> Hit send too soon!
>
> By the way, the bug seems to be exposed only with a certain pattern/sequence
> of workers being detached (perhaps in immediate successive) whereby the
> funnel->nextqueue remains incorrectly set.
>
> The patch attached this time.
>
> By the way, when I have asserts enabled, I hit this compilation error:
>
> createplan.c: In function ‘create_partialseqscan_plan’:
> createplan.c:1180: error: ‘Path’ has no member named ‘path’
>
> I see following line there:
>
> Assert(best_path->path.parent->rtekind == RTE_RELATION);
>
Okay, will take care of this.
Attachment
>
>
> Another problem. I restarted the instance (just in case), and get this error:
>
> # \df+ *.*
> ERROR: cannot retain locks acquired while in parallel mode
>
>
> On Tue, Mar 10, 2015 at 10:23 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote:
> >
> > On Tue, Mar 10, 2015 at 3:09 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > > I have currently modelled it based on existing rescan for seqscan
> > > (ExecReScanSeqScan()) which means it will begin the scan again.
> > > Basically if the workers are already started/initialized by previous
> > > scan, then re-initialize them (refer function ExecReScanFunnel() in
> > > patch).
> > >
> > > Can you elaborate more if you think current handling is not sufficient
> > > for any case?
> >
> > From ExecReScanFunnel function it seems that the re-scan waits till
> > all the workers
> > has to be finished to start again the next scan. Are the workers will
> > stop the current
> > ongoing task? otherwise this may decrease the performance instead of
> > improving as i feel.
> >
>
> Okay, performance-wise it might effect such a case, but I think we can
> handle it by not calling WaitForParallelWorkersToFinish(),
> as DestroyParallelContext() will automatically terminate all the workers.
>
We can't directly call DestroyParallelContext() to terminate workers as
>
> In create_parallelscan_paths() function the funnel path is added once
> the partial seq scan
> path is generated. I feel the funnel path can be added once on top of
> the total possible
> parallel path in the entire query path.
>
> Is this the right patch to add such support also?
>
This seems to be an optimization for parallel paths which can be
On Fri, Mar 13, 2015 at 9:01 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Mar 12, 2015 at 3:44 AM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> In create_parallelscan_paths() function the funnel path is added once >> the partial seq scan >> path is generated. I feel the funnel path can be added once on top of >> the total possible >> parallel path in the entire query path. >> >> Is this the right patch to add such support also? > > This seems to be an optimization for parallel paths which can be > done later as well. +1. Let's keep it simple for now. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Mar 13, 2015 at 8:59 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > We can't directly call DestroyParallelContext() to terminate workers as > it can so happen that by that time some of the workers are still not > started. That shouldn't be a problem. TerminateBackgroundWorker() not only kills an existing worker if there is one, but also tells the postmaster that if it hasn't started the worker yet, it should not bother. So at the conclusion of the first loop inside DestroyParallelContext(), every running worker will have received SIGTERM and no more workers will be started. > So that can lead to problem. I think what we need here is a way to know > whether all workers are started. (basically need a new function >WaitForParallelWorkersToStart()). This API needs to be provided by > parallel-mode patch. I don't think so. DestroyParallelContext() is intended to be good enough for this purpose; if it's not, we should fix that instead of adding a new function. No matter what, re-scanning a parallel node is not going to be very efficient. But the way to deal with that is to make sure that such nodes have a substantial startup cost, so that the planner won't pick them in the case where it isn't going to work out well. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Mar 13, 2015 at 7:01 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I think this can happen if funnel->nextqueue is greater than > funnel->nqueues. > Please see if attached patch fixes the issue, else could you share the > scenario in more detail where you hit this issue. Speaking as the guy who wrote the first version of that code... I don't think this is the right fix; the point of that code is to remove a tuple queue from the funnel when it gets detached, which is a correct thing to want to do. funnel->nextqueue should always be less than funnel->nqueues; how is that failing to be the case here? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Mar 13, 2015 at 7:01 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I think this can happen if funnel->nextqueue is greater than
> > funnel->nqueues.
> > Please see if attached patch fixes the issue, else could you share the
> > scenario in more detail where you hit this issue.
>
> Speaking as the guy who wrote the first version of that code...
>
> I don't think this is the right fix; the point of that code is to
> remove a tuple queue from the funnel when it gets detached, which is a
> correct thing to want to do. funnel->nextqueue should always be less
> than funnel->nqueues; how is that failing to be the case here?
>
I could not reproduce the issue, neither the exact scenario is
On Fri, Mar 13, 2015 at 11:03 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Mar 13, 2015 at 7:15 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> On Fri, Mar 13, 2015 at 7:01 AM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > I think this can happen if funnel->nextqueue is greater than >> > funnel->nqueues. >> > Please see if attached patch fixes the issue, else could you share the >> > scenario in more detail where you hit this issue. >> >> Speaking as the guy who wrote the first version of that code... >> >> I don't think this is the right fix; the point of that code is to >> remove a tuple queue from the funnel when it gets detached, which is a >> correct thing to want to do. funnel->nextqueue should always be less >> than funnel->nqueues; how is that failing to be the case here? >> > > I could not reproduce the issue, neither the exact scenario is > mentioned in mail. However what I think can lead to funnel->nextqueue > greater than funnel->nqueues is something like below: > > Assume 5 queues, so value of funnel->nqueues will be 5 and > assume value of funnel->nextqueue is 2, so now let us say 4 workers > got detached one-by-one, so for such a case it will always go in else loop > and will never change funnel->nextqueue whereas value of funnel->nqueues > will become 1. > > Am I missing something? > Sorry, I did not mention the exact example I'd used but I thought it was just any arbitrary example: CREATE TABLE t1(c1, c2) SELECT g1, repeat('x', 5) FROM generate_series(1, 10000000) g; CREATE TABLE t2(c1, c2) SELECT g1, repeat('x', 5) FROM generate_series(1, 1000000) g; SELECT count(*) FROM t1 JOIN t2 ON t1.c1 = t2.c1 AND t1.c1 BETWEEN 100 AND 200; The observed behavior included a hang or segfault arbitrarily (that's why I guessed it may be arbitrariness of sequence of detachment of workers). Changed parameters to cause plan to include a Funnel: parallel_seqscan_degree = 8 cpu_tuple_communication_cost = 0.01/0.001 Thanks, Amit
>
> On 12 March 2015 at 15:29, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Please note that parallel_setup_cost and parallel_startup_cost are
> > still set to zero by default, so you need to set it to higher values
> > if you don't want the parallel plans once parallel_seqscan_degree
> > is set. I have yet to comeup with default values for them, needs
> > some tests.
>
> Thanks. Getting a problem:
>
>
> # SELECT name, setting FROM pg_settings WHERE name IN
> ('parallel_seqscan_degree','max_worker_processes','seq_page_cost');
> name | setting
> -------------------------+---------
> max_worker_processes | 20
> parallel_seqscan_degree | 8
> seq_page_cost | 1000
> (3 rows)
>
> # EXPLAIN SELECT DISTINCT bid FROM pgbench_accounts;
> ERROR: too many dynamic shared memory segments
>
>
Problem-2:
On 13-03-2015 PM 11:03, Amit Kapila wrote: > On Fri, Mar 13, 2015 at 7:15 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> I don't think this is the right fix; the point of that code is to >> remove a tuple queue from the funnel when it gets detached, which is a >> correct thing to want to do. funnel->nextqueue should always be less >> than funnel->nqueues; how is that failing to be the case here? >> > > I could not reproduce the issue, neither the exact scenario is > mentioned in mail. However what I think can lead to funnel->nextqueue > greater than funnel->nqueues is something like below: > > Assume 5 queues, so value of funnel->nqueues will be 5 and > assume value of funnel->nextqueue is 2, so now let us say 4 workers > got detached one-by-one, so for such a case it will always go in else loop > and will never change funnel->nextqueue whereas value of funnel->nqueues > will become 1. > Or if the just-detached queue happens to be the last one, we'll make shm_mq_receive() to read from a potentially already-detached queue in the immediately next iteration. That seems to be caused by not having updated the funnel->nextqueue. With the returned value being SHM_MQ_DETACHED, we'll again try to remove it from the queue. In this case, it causes the third argument to memcpy be negative and hence the segfault. I can't seem to really figure out the other problem of waiting forever in WaitLatch() but I had managed to make it go away with: - if (funnel->nextqueue == waitpos) + if (result != SHM_MQ_DETACHED && funnel->nextqueue == waitpos) By the way, you can try reproducing this with the example I posted on Friday. Thanks, Amit
>
> On 13-03-2015 PM 11:03, Amit Kapila wrote:
> > On Fri, Mar 13, 2015 at 7:15 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >>
> >> I don't think this is the right fix; the point of that code is to
> >> remove a tuple queue from the funnel when it gets detached, which is a
> >> correct thing to want to do. funnel->nextqueue should always be less
> >> than funnel->nqueues; how is that failing to be the case here?
> >>
> >
> > I could not reproduce the issue, neither the exact scenario is
> > mentioned in mail. However what I think can lead to funnel->nextqueue
> > greater than funnel->nqueues is something like below:
> >
> > Assume 5 queues, so value of funnel->nqueues will be 5 and
> > assume value of funnel->nextqueue is 2, so now let us say 4 workers
> > got detached one-by-one, so for such a case it will always go in else loop
> > and will never change funnel->nextqueue whereas value of funnel->nqueues
> > will become 1.
> >
>
> Or if the just-detached queue happens to be the last one, we'll make
> shm_mq_receive() to read from a potentially already-detached queue in the
> immediately next iteration.
{
--funnel->nqueues;
if (funnel->nqueues == 0)
{
if (done != NULL)
*done = true;
return NULL;
}
> funnel->nextqueue. With the returned value being SHM_MQ_DETACHED, we'll again
> try to remove it from the queue. In this case, it causes the third argument to
>
> I can't seem to really figure out the other problem of waiting forever in
> WaitLatch()
> By the way, you can try reproducing this with the example I posted on Friday.
>
>
> Amit Kapila wrote:
>
> > I think this can happen if funnel->nextqueue is greater
> > than funnel->nqueues.
> > Please see if attached patch fixes the issue, else could you share the
> > scenario in more detail where you hit this issue.
>
> Uh, isn't this copying an overlapping memory region? If so you should
> be using memmove instead.
>
On 16-03-2015 PM 04:14, Amit Kapila wrote: > On Mon, Mar 16, 2015 at 9:40 AM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> > wrote: >> Or if the just-detached queue happens to be the last one, we'll make >> shm_mq_receive() to read from a potentially already-detached queue in the >> immediately next iteration. > > Won't the last queue case already handled by below code: > else > { > --funnel->nqueues; > if (funnel->nqueues == 0) > { > if (done != NULL) > *done = true; > return NULL; > } > Actually I meant "currently the last" or: funnel->nextqueue == funnel->nqueue - 1 So the code you quote would only take care of subset of the cases. Imagine funnel->nqueues going down from 5 to 3 in successive iterations while funnel->nextqueue remains set to 4 (which would have been the "currently last" when funnel->nqueues was 5). >> I can't seem to really figure out the other problem of waiting forever in >> WaitLatch() >> > > The reason seems that for certain scenarios, the way we set the latch before > exiting needs some more thought. Currently we are setting the latch in > HandleParallelMessageInterrupt(), that doesn't seem to be sufficient. > How about shm_mq_detach() called from ParallelQueryMain() right after exec_parallel_stmt() returns? Doesn't that do the SetLatch() that needs to be done by a worker? Thanks, Amit
>
> On Fri, Mar 13, 2015 at 8:59 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > We can't directly call DestroyParallelContext() to terminate workers as
> > it can so happen that by that time some of the workers are still not
> > started.
>
> That shouldn't be a problem. TerminateBackgroundWorker() not only
> kills an existing worker if there is one, but also tells the
> postmaster that if it hasn't started the worker yet, it should not
> bother. So at the conclusion of the first loop inside
> DestroyParallelContext(), every running worker will have received
> SIGTERM and no more workers will be started.
>
+ if (status == BGWH_STOPPED)
+ return status;
On Tue, Mar 17, 2015 at 1:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > The problem occurs in second loop inside DestroyParallelContext() > where it calls WaitForBackgroundWorkerShutdown(). Basically > WaitForBackgroundWorkerShutdown() just checks for BGWH_STOPPED > status, refer below code in parallel-mode patch: > > + status = GetBackgroundWorkerPid(handle, &pid); > + if (status == BGWH_STOPPED) > + return status; > > So if the status here returned is BGWH_NOT_YET_STARTED, then it > will go for WaitLatch and will there forever. > > I think fix is to check if status is BGWH_STOPPED or BGWH_NOT_YET_STARTED, > then just return the status. > > What do you say? No, that's not right. If we return when the status is BGWH_NOT_YET_STARTED, then the postmaster could subsequently start the worker. Can you try this: diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c index f80141a..39b919f 100644 --- a/src/backend/postmaster/bgworker.c +++ b/src/backend/postmaster/bgworker.c @@ -244,6 +244,8 @@ BackgroundWorkerStateChange(void) rw->rw_terminate = true; if (rw->rw_pid != 0) kill(rw->rw_pid, SIGTERM); + else + ReportBackgroundWorkerPID(rw); } continue; } -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Tue, Mar 17, 2015 at 1:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > The problem occurs in second loop inside DestroyParallelContext()
> > where it calls WaitForBackgroundWorkerShutdown(). Basically
> > WaitForBackgroundWorkerShutdown() just checks for BGWH_STOPPED
> > status, refer below code in parallel-mode patch:
> >
> > + status = GetBackgroundWorkerPid(handle, &pid);
> > + if (status == BGWH_STOPPED)
> > + return status;
> >
> > So if the status here returned is BGWH_NOT_YET_STARTED, then it
> > will go for WaitLatch and will there forever.
> >
> > I think fix is to check if status is BGWH_STOPPED or BGWH_NOT_YET_STARTED,
> > then just return the status.
> >
> > What do you say?
>
> No, that's not right. If we return when the status is
> BGWH_NOT_YET_STARTED, then the postmaster could subsequently start the
> worker.
>
> Can you try this:
>
> diff --git a/src/backend/postmaster/bgworker.c
> b/src/backend/postmaster/bgworker.c
> index f80141a..39b919f 100644
> --- a/src/backend/postmaster/bgworker.c
> +++ b/src/backend/postmaster/bgworker.c
> @@ -244,6 +244,8 @@ BackgroundWorkerStateChange(void)
> rw->rw_terminate = true;
> if (rw->rw_pid != 0)
> kill(rw->rw_pid, SIGTERM);
> + else
> + ReportBackgroundWorkerPID(rw);
> }
> continue;
> }
>
It didn't fix the problem. IIUC, you have done this to ensure that
On Wed, Mar 18, 2015 at 2:22 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Can you try this: >> >> diff --git a/src/backend/postmaster/bgworker.c >> b/src/backend/postmaster/bgworker.c >> index f80141a..39b919f 100644 >> --- a/src/backend/postmaster/bgworker.c >> +++ b/src/backend/postmaster/bgworker.c >> @@ -244,6 +244,8 @@ BackgroundWorkerStateChange(void) >> rw->rw_terminate = true; >> if (rw->rw_pid != 0) >> kill(rw->rw_pid, SIGTERM); >> + else >> + ReportBackgroundWorkerPID(rw); >> } >> continue; >> } >> > > It didn't fix the problem. IIUC, you have done this to ensure that > if worker is not already started, then update it's pid, so that we > can get the required status in WaitForBackgroundWorkerShutdown(). > As this is a timing issue, it can so happen that before Postmaster > gets a chance to report the pid, backend has already started waiting > on WaitLatch(). I think I figured out the problem. That fix only helps in the case where the postmaster noticed the new registration previously but didn't start the worker, and then later notices the termination. What's much more likely to happen is that the worker is started and terminated so quickly that both happen before we create a RegisteredBgWorker for it. The attached patch fixes that case, too. Assuming this actually fixes the problem, I think we should back-patch it into 9.4. To recap, the problem is that, at present, if you register a worker and then terminate it before it's launched, GetBackgroundWorkerPid() will still return BGWH_NOT_YET_STARTED, which it makes it seem like we're still waiting for it to start. But when or if the slot is reused for an unrelated registration, then GetBackgroundWorkerPid() will switch to returning BGWH_STOPPED. It's hard to believe that's the behavior anyone wants. With this patch, the return value will always be BGWH_STOPPED in this situation. That has the virtue of being consistent, and practically speaking I think it's the behavior that everyone will want, because the case where this matters is when you are waiting for workers to start or waiting for worker to stop, and in either case you will want to treat a worker that was marked for termination before the postmaster actually started it as already-stopped rather than not-yet-started. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
On Sat, Mar 14, 2015 at 1:04 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> # EXPLAIN SELECT DISTINCT bid FROM pgbench_accounts; >> ERROR: too many dynamic shared memory segments > > This happens because we have maximum limit on the number of > dynamic shared memory segments in the system. > > In function dsm_postmaster_startup(), it is defined as follows: > > maxitems = PG_DYNSHMEM_FIXED_SLOTS > + PG_DYNSHMEM_SLOTS_PER_BACKEND * MaxBackends; > > In the above case, it is choosing parallel plan for each of the > AppendRelation, > (because of seq_page_cost = 1000) and that causes the test to > cross max limit of dsm segments. The problem here is, of course, that each parallel sequential scan is trying to create an entirely separate group of workers. Eventually, I think we should fix this by rejiggering things so that when there are multiple parallel nodes in a plan, they all share a pool of workers. So each worker would actually get a list of plan nodes instead of a single plan node. Maybe it works on the first node in the list until that's done, and then moves onto the next, or maybe it round-robins among all the nodes and works on the ones where the output tuple queues aren't currently full, or maybe the master somehow notifies the workers which nodes are most useful to work on at the present time. But I think trying to figure this out is far too ambitious for 9.5, and I think we can have a useful feature without implementing any of it. But, we can't just ignore the issue right now, because erroring out on a large inheritance hierarchy is no good. Instead, we should fall back to non-parallel operation in this case. By the time we discover the problem, it's too late to change the plan, because it's already execution time. So we are going to be stuck executing the parallel node - just with no workers to help. However, what I think we can do is use a slab of backend-private memory instead of a dynamic shared memory segment, and in that way avoid this error. We do something similar when starting the postmaster in stand-alone mode: the main shared memory segment is replaced by a backend-private allocation with the same contents that the shared memory segment would normally have. The same fix will work here. Even once we make the planner and executor smarter, so that they don't create lots of shared memory segments and lots of separate worker pools in this type of case, it's probably still useful to have this as a fallback approach, because there's always the possibility that some other client of the dynamic shared memory system could gobble up all the segments. So, I'm going to go try to figure out the best way to implement this. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Sat, Mar 14, 2015 at 1:04 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> # EXPLAIN SELECT DISTINCT bid FROM pgbench_accounts;
> >> ERROR: too many dynamic shared memory segments
> >
> > This happens because we have maximum limit on the number of
> > dynamic shared memory segments in the system.
> >
> > In function dsm_postmaster_startup(), it is defined as follows:
> >
> > maxitems = PG_DYNSHMEM_FIXED_SLOTS
> > + PG_DYNSHMEM_SLOTS_PER_BACKEND * MaxBackends;
> >
> > In the above case, it is choosing parallel plan for each of the
> > AppendRelation,
> > (because of seq_page_cost = 1000) and that causes the test to
> > cross max limit of dsm segments.
>
> The problem here is, of course, that each parallel sequential scan is
> trying to create an entirely separate group of workers. Eventually, I
> think we should fix this by rejiggering things so that when there are
> multiple parallel nodes in a plan, they all share a pool of workers.
> So each worker would actually get a list of plan nodes instead of a
> single plan node. Maybe it works on the first node in the list until
> that's done, and then moves onto the next, or maybe it round-robins
> among all the nodes and works on the ones where the output tuple
> queues aren't currently full, or maybe the master somehow notifies the
> workers which nodes are most useful to work on at the present time.
> But I think trying to figure this out is far too ambitious for 9.5,
> and I think we can have a useful feature without implementing any of
> it.
>
> But, we can't just ignore the issue right now, because erroring out on
> a large inheritance hierarchy is no good. Instead, we should fall
> back to non-parallel operation in this case. By the time we discover
> the problem, it's too late to change the plan, because it's already
> execution time. So we are going to be stuck executing the parallel
> node - just with no workers to help. However, what I think we can do
> is use a slab of backend-private memory instead of a dynamic shared
> memory segment, and in that way avoid this error. We do something
> similar when starting the postmaster in stand-alone mode: the main
> shared memory segment is replaced by a backend-private allocation with
> the same contents that the shared memory segment would normally have.
> The same fix will work here.
>
> Even once we make the planner and executor smarter, so that they don't
> create lots of shared memory segments and lots of separate worker
> pools in this type of case, it's probably still useful to have this as
> a fallback approach, because there's always the possibility that some
> other client of the dynamic shared memory system could gobble up all
> the segments. So, I'm going to go try to figure out the best way to
> implement this.
>
Thanks.
>
> On Wed, Mar 18, 2015 at 2:22 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > It didn't fix the problem. IIUC, you have done this to ensure that
> > if worker is not already started, then update it's pid, so that we
> > can get the required status in WaitForBackgroundWorkerShutdown().
> > As this is a timing issue, it can so happen that before Postmaster
> > gets a chance to report the pid, backend has already started waiting
> > on WaitLatch().
>
> I think I figured out the problem. That fix only helps in the case
> where the postmaster noticed the new registration previously but
> didn't start the worker, and then later notices the termination.
> What's much more likely to happen is that the worker is started and
> terminated so quickly that both happen before we create a
> RegisteredBgWorker for it. The attached patch fixes that case, too.
>
> Assuming this actually fixes the problem, I think we should back-patch
> it into 9.4.
On Wed, Mar 18, 2015 at 11:43 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Patch fixes the problem and now for Rescan, we don't need to Wait > for workers to finish. > >> Assuming this actually fixes the problem, I think we should back-patch >> it into 9.4. > > +1 OK, done. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On 16-03-2015 PM 04:14, Amit Kapila wrote:
> > On Mon, Mar 16, 2015 at 9:40 AM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp>
> > wrote:
> >> Or if the just-detached queue happens to be the last one, we'll make
> >> shm_mq_receive() to read from a potentially already-detached queue in the
> >> immediately next iteration.
> >
> > Won't the last queue case already handled by below code:
> > else
> > {
> > --funnel->nqueues;
> > if (funnel->nqueues == 0)
> > {
> > if (done != NULL)
> > *done = true;
> > return NULL;
> > }
> >
>
> Actually I meant "currently the last" or:
>
> funnel->nextqueue == funnel->nqueue - 1
>
> So the code you quote would only take care of subset of the cases.
>
Fixed this issue by resetting funnel->next queue to zero (as per offlist
>
> >> I can't seem to really figure out the other problem of waiting forever in
> >> WaitLatch()
> >>
> >
> > The reason seems that for certain scenarios, the way we set the latch before
> > exiting needs some more thought. Currently we are setting the latch in
> > HandleParallelMessageInterrupt(), that doesn't seem to be sufficient.
> >
>
> How about shm_mq_detach() called from ParallelQueryMain() right after
> exec_parallel_stmt() returns? Doesn't that do the SetLatch() that needs to be
> done by a worker?
>
Attachment
On 20-03-2015 PM 09:06, Amit Kapila wrote: > On Mon, Mar 16, 2015 at 12:58 PM, Amit Langote < > Langote_Amit_f8@lab.ntt.co.jp> wrote: >> Actually I meant "currently the last" or: >> >> funnel->nextqueue == funnel->nqueue - 1 >> >> So the code you quote would only take care of subset of the cases. >> > > Fixed this issue by resetting funnel->next queue to zero (as per offlist > discussion with Robert), so that it restarts from first queue in such > a case. > >> >> >> How about shm_mq_detach() called from ParallelQueryMain() right after >> exec_parallel_stmt() returns? Doesn't that do the SetLatch() that needs > to be >> done by a worker? >> > > Fixed this issue by not going for Wait incase of detached queues. > Thanks for fixing. I no longer see the problems. Regards, Amit
On 20 March 2015 17:37, Amit Kapila Wrote:
> So the patches have to be applied in below sequence:
> HEAD Commit-id : 8d1f2390
> parallel-mode-v8.1.patch [2]
> assess-parallel-safety-v4.patch [1]
> parallel-heap-scan.patch [3]
> parallel_seqscan_v11.patch (Attached with this mail)
While I was going through this patch, I observed one invalid ASSERT in the function “ExecInitFunnel” i.e.
Assert(outerPlan(node) == NULL);
Outer node of Funnel node is always non-NULL and currently it will be PartialSeqScan Node.
May be ASSERT is disabled while building the code because of which this issue has not yet been observed.
Thanks and Regards,
Kumar Rajeev Rastogi
>
>
> So the patches have to be applied in below sequence:
> HEAD Commit-id : 8d1f2390
> parallel-mode-v8.1.patch [2]
> assess-parallel-safety-v4.patch [1]
> parallel-heap-scan.patch [3]
> parallel_seqscan_v11.patch (Attached with this mail)
>
> The reason for not using the latest commit in HEAD is that latest
> version of assess-parallel-safety patch was not getting applied,
> so I generated the patch at commit-id where I could apply that
> patch successfully.
>
> [1] - http://www.postgresql.org/message-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
> [2] - http://www.postgresql.org/message-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
> [3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
>
assess-parallel-safety-v4.patch [1]
[2] - http://www.postgresql.org/message-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
Attachment
>
> On 20 March 2015 17:37, Amit Kapila Wrote:
>
> > So the patches have to be applied in below sequence:
> > HEAD Commit-id : 8d1f2390
> > parallel-mode-v8.1.patch [2]
> > assess-parallel-safety-v4.patch [1]
> > parallel-heap-scan.patch [3]
> > parallel_seqscan_v11.patch (Attached with this mail)
>
> While I was going through this patch, I observed one invalid ASSERT in the function “ExecInitFunnel” i.e.
>
> Assert(outerPlan(node) == NULL);
>
> Outer node of Funnel node is always non-NULL and currently it will be PartialSeqScan Node.
>
+ Assert(innerPlan(node) == NULL);
On 25 March 2015 16:00, Amit Kapila Wrote:
> Which version of patch you are looking at?
> I am seeing below code in ExecInitFunnel() in Version-11 to which
> you have replied.
> + /* Funnel node doesn't have innerPlan node. */
> + Assert(innerPlan(node) == NULL
I was seeing the version-10.
I just checked version-11 and version-12 and found to be already fixed.
I should have checked the latest version before sending the report…J
Thanks and Regards,
Kumar Rajeev Rastogi
From: Amit Kapila [mailto:amit.kapila16@gmail.com]
Sent: 25 March 2015 16:00
To: Rajeev rastogi
Cc: Amit Langote; Robert Haas; Andres Freund; Kouhei Kaigai; Amit Langote; Fabrízio Mello; Thom Brown; Stephen Frost; pgsql-hackers
Subject: Re: [HACKERS] Parallel Seq Scan
On Wed, Mar 25, 2015 at 3:47 PM, Rajeev rastogi <rajeev.rastogi@huawei.com> wrote:
>
> On 20 March 2015 17:37, Amit Kapila Wrote:
>
> > So the patches have to be applied in below sequence:
> > HEAD Commit-id : 8d1f2390
> > parallel-mode-v8.1.patch [2]
> > assess-parallel-safety-v4.patch [1]
> > parallel-heap-scan.patch [3]
> > parallel_seqscan_v11.patch (Attached with this mail)
>
> While I was going through this patch, I observed one invalid ASSERT in the function “ExecInitFunnel” i.e.
>
> Assert(outerPlan(node) == NULL);
>
> Outer node of Funnel node is always non-NULL and currently it will be PartialSeqScan Node.
>
Which version of patch you are looking at?
I am seeing below code in ExecInitFunnel() in Version-11 to which
you have replied.
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
>
> On 25 March 2015 16:00, Amit Kapila Wrote:
>
> > Which version of patch you are looking at?
>
> > I am seeing below code in ExecInitFunnel() in Version-11 to which
>
> > you have replied.
>
>
>
> > + /* Funnel node doesn't have innerPlan node. */
> > + Assert(innerPlan(node) == NULL
>
>
>
> I was seeing the version-10.
>
> I just checked version-11 and version-12 and found to be already fixed.
>
> I should have checked the latest version before sending the report…J
>
No problem, Thanks for looking into the patch.
On Fri, Mar 20, 2015 at 5:36 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
>
> So the patches have to be applied in below sequence:
> HEAD Commit-id : 8d1f2390
> parallel-mode-v8.1.patch [2]
> assess-parallel-safety-v4.patch [1]
> parallel-heap-scan.patch [3]
> parallel_seqscan_v11.patch (Attached with this mail)
>
> The reason for not using the latest commit in HEAD is that latest
> version of assess-parallel-safety patch was not getting applied,
> so I generated the patch at commit-id where I could apply that
> patch successfully.
>
> [1] - http://www.postgresql.org/message-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
> [2] - http://www.postgresql.org/message-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
> [3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
>Fixed the reported issue on assess-parallel-safety thread and anotherbug caught while testing joins and integrated with latest version ofparallel-mode patch (parallel-mode-v9 patch).Apart from that I have moved the Initialization of dsm segement fromInitNode phase to ExecFunnel() (on first execution) as per suggestionfrom Robert. The main idea is that as it creates large shared memorysegment, so do the work when it is really required.HEAD Commit-Id: 11226e38parallel-mode-v9.patch [2]
assess-parallel-safety-v4.patch [1]parallel-heap-scan.patch [3]parallel_seqscan_v12.patch (Attached with this mail)[1] - http://www.postgresql.org/message-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] - http://www.postgresql.org/message-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
grep -r 'starting background worker process "parallel worker for PID 12165"' postgresql-2015-03-25_112522.log | wc -l
2496
QUERY PLAN
---------------------------------------------------------------------------------------------------------
HashAggregate (cost=38856527.50..38856529.50 rows=200 width=4)
Group Key: pgbench_accounts.bid
-> Append (cost=0.00..38806370.00 rows=20063001 width=4)
-> Seq Scan on pgbench_accounts (cost=0.00..0.00 rows=1 width=4)
-> Funnel on pgbench_accounts_1 (cost=0.00..192333.33 rows=100000 width=4)
Number of Workers: 8
-> Partial Seq Scan on pgbench_accounts_1 (cost=0.00..1641000.00 rows=100000 width=4)
-> Funnel on pgbench_accounts_2 (cost=0.00..192333.33 rows=100000 width=4)
Number of Workers: 8
-> Partial Seq Scan on pgbench_accounts_2 (cost=0.00..1641000.00 rows=100000 width=4)
-> Funnel on pgbench_accounts_3 (cost=0.00..192333.33 rows=100000 width=4)
Number of Workers: 8
...
-> Partial Seq Scan on pgbench_accounts_498 (cost=0.00..10002.10 rows=210 width=4)
-> Funnel on pgbench_accounts_499 (cost=0.00..1132.34 rows=210 width=4)
Number of Workers: 8
-> Partial Seq Scan on pgbench_accounts_499 (cost=0.00..10002.10 rows=210 width=4)
-> Funnel on pgbench_accounts_500 (cost=0.00..1132.34 rows=210 width=4)
Number of Workers: 8
-> Partial Seq Scan on pgbench_accounts_500 (cost=0.00..10002.10 rows=210 width=4)
--
Still not sure why 8 workers are needed for each partial scan. I would expect 8 workers to be used for 8 separate scans. Perhaps this is just my misunderstanding of how this feature works.
2015-03-25 13:17:49 GMT [22823]: [124-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [125-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [126-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [127-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [128-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [129-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [130-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [131-1] user=,db=,client= LOG: registering background worker "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [132-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [133-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [134-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [135-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [136-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [137-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [138-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [139-1] user=,db=,client= LOG: starting background worker process "parallel worker for PID 24792"
2015-03-25 13:17:49 GMT [22823]: [140-1] user=,db=,client= LOG: worker process: parallel worker for PID 24792 (PID 24804) was terminated by signal 11: Segmentation fault
2015-03-25 13:17:49 GMT [22823]: [141-1] user=,db=,client= LOG: terminating any other active server processes
2015-03-25 13:17:49 GMT [24777]: [2-1] user=,db=,client= WARNING: terminating connection because of crash of another server process
2015-03-25 13:17:49 GMT [24777]: [3-1] user=,db=,client= DETAIL: The postmaster has commanded this server process to roll back the current transaction and exit, because another server process exited abnormally and possibly corrupted shared memory.
2015-03-25 13:17:49 GMT [24777]: [4-1] user=,db=,client= HINT: In a moment you should be able to reconnect to the database and repeat your command.
#0 GrantLockLocal (locallock=locallock@entry=0xfbe7f0, owner=owner@entry=0x1046da0) at lock.c:1544
#1 0x000000000066975c in LockAcquireExtended (locktag=locktag@entry=0x7fffdcb0ea20, lockmode=1,
lockmode@entry=<error reading variable: Cannot access memory at address 0x7fffdcb0e9f0>, sessionLock=sessionLock@entry=0 '\000', dontWait=dontWait@entry=0 '\000',
reportMemoryError=reportMemoryError@entry=1 '\001', ) at lock.c:798
#2 0x000000000066a1c4 in LockAcquire (locktag=locktag@entry=0x7fffdcb0ea20, lockmode=<error reading variable: Cannot access memory at address 0x7fffdcb0e9f0>,
sessionLock=sessionLock@entry=0 '\000', dontWait=dontWait@entry=0 '\000') at lock.c:680
#3 0x0000000000667c48 in LockRelationOid (relid=<error reading variable: Cannot access memory at address 0x7fffdcb0e9e8>,
relid@entry=<error reading variable: Cannot access memory at address 0x7fffdcb0ea48>,
lockmode=<error reading variable: Cannot access memory at address 0x7fffdcb0e9f0>,
lockmode@entry=<error reading variable: Cannot access memory at address 0x7fffdcb0ea48>) at lmgr.c:94
#0 hash_search_with_hash_value (hashp=0x2a2c370, keyPtr=keyPtr@entry=0x7ffff5ad2230, hashvalue=hashvalue@entry=2114233864, action=action@entry=HASH_FIND,
foundPtr=foundPtr@entry=0x0) at dynahash.c:918
#1 0x0000000000654d1a in BufTableLookup (tagPtr=tagPtr@entry=0x7ffff5ad2230, hashcode=hashcode@entry=2114233864) at buf_table.c:96
#2 0x000000000065746b in BufferAlloc (foundPtr=0x7ffff5ad222f <Address 0x7ffff5ad222f out of bounds>, strategy=0x0,
blockNum=<error reading variable: Cannot access memory at address 0x7ffff5ad2204>,
forkNum=<error reading variable: Cannot access memory at address 0x7ffff5ad2208>,
relpersistence=<error reading variable: Cannot access memory at address 0x7ffff5ad2214>, smgr=0x2aaae00) at bufmgr.c:893
#3 ReadBuffer_common (smgr=0x2aaae00, relpersistence=<optimized out>, ) at bufmgr.c:641
#4 0x0000000000657e40 in ReadBufferExtended (reln=<error reading variable: Cannot access memory at address 0x7ffff5ad2278>,
reln@entry=<error reading variable: Cannot access memory at address 0x7ffff5ad22f8>, forkNum=MAIN_FORKNUM, blockNum=6, mode=<optimized out>,
strategy=<optimized out>) at bufmgr.c:560
#0 hash_search_with_hash_value (hashp=0x1d97370, keyPtr=keyPtr@entry=0x7ffff95855f0, hashvalue=hashvalue@entry=2382868486, action=action@entry=HASH_FIND,
foundPtr=foundPtr@entry=0x0) at dynahash.c:907
#1 0x0000000000654d1a in BufTableLookup (tagPtr=tagPtr@entry=0x7ffff95855f0, hashcode=hashcode@entry=2382868486) at buf_table.c:96
#2 0x000000000065746b in BufferAlloc (foundPtr=0x7ffff95855ef "", strategy=0x0, blockNum=9, forkNum=MAIN_FORKNUM, relpersistence=112 'p', smgr=0x1e15860)
at bufmgr.c:893
#3 ReadBuffer_common (smgr=0x1e15860, relpersistence=<optimized out>, forkNum=forkNum@entry=MAIN_FORKNUM, blockNum=blockNum@entry=9, mode=RBM_NORMAL, strategy=0x0,
hit=hit@entry=0x7ffff958569f "") at bufmgr.c:641
#4 0x0000000000657e40 in ReadBufferExtended (reln=reln@entry=0x7f8a17bab2c0, forkNum=forkNum@entry=MAIN_FORKNUM, blockNum=9, mode=mode@entry=RBM_NORMAL,
strategy=strategy@entry=0x0) at bufmgr.c:560
#5 0x0000000000657f4d in ReadBuffer (blockNum=<optimized out>, reln=0x7f8a17bab2c0) at bufmgr.c:492
#6 ReleaseAndReadBuffer (buffer=buffer@entry=398111424, relation=relation@entry=0x1, blockNum=<optimized out>) at bufmgr.c:1403
#7 0x000000000049e6bf in _bt_relandgetbuf (rel=0x1, rel@entry=0x7f8a17bab2c0, obuf=398111424, blkno=blkno@entry=9, access=access@entry=1) at nbtpage.c:707
#8 0x00000000004a24b4 in _bt_search (rel=rel@entry=0x7f8a17bab2c0, keysz=keysz@entry=2, scankey=scankey@entry=0x7ffff95858b0, nextkey=nextkey@entry=0 '\000',
bufP=bufP@entry=0x7ffff95857ac, access=access@entry=1) at nbtsearch.c:131
#9 0x00000000004a2cb4 in _bt_first (scan=scan@entry=0x1eb2048, dir=dir@entry=ForwardScanDirection) at nbtsearch.c:940
#10 0x00000000004a1141 in btgettuple (fcinfo=<optimized out>) at nbtree.c:288
#11 0x0000000000759132 in FunctionCall2Coll (flinfo=flinfo@entry=0x1e34390, collation=collation@entry=0, arg1=arg1@entry=32186440, arg2=arg2@entry=1) at fmgr.c:1323
#12 0x000000000049b273 in index_getnext_tid (scan=scan@entry=0x1eb2048, direction=direction@entry=ForwardScanDirection) at indexam.c:462
#13 0x000000000049b450 in index_getnext (scan=0x1eb2048, direction=direction@entry=ForwardScanDirection) at indexam.c:602
#14 0x000000000049a9a9 in systable_getnext (sysscan=sysscan@entry=0x1eb1ff8) at genam.c:416
#15 0x0000000000740452 in SearchCatCache (cache=0x1ddf540, v1=<optimized out>, v2=<optimized out>, v3=<optimized out>, v4=<optimized out>) at catcache.c:1248
#16 0x000000000074bd06 in GetSysCacheOid (cacheId=cacheId@entry=44, key1=key1@entry=140226851237264, key2=<optimized out>, key3=key3@entry=0, key4=key4@entry=0)
at syscache.c:988
#17 0x000000000074d674 in get_relname_relid (relname=relname@entry=0x7f891ba7ed90 "pgbench_accounts_3", relnamespace=<optimized out>) at lsyscache.c:1602
#18 0x00000000004e1228 in RelationIsVisible (relid=relid@entry=16428) at namespace.c:740
#19 0x00000000004e4b6f in pg_table_is_visible (fcinfo=0x1e9dfc8) at namespace.c:4078
#20 0x0000000000595f72 in ExecMakeFunctionResultNoSets (fcache=0x1e9df58, econtext=0x1e99848, isNull=0x7ffff95871bf "", isDone=<optimized out>) at execQual.c:2015
#21 0x000000000059b469 in ExecQual (qual=qual@entry=0x1e9b368, econtext=econtext@entry=0x1e99848, resultForNull=resultForNull@entry=0 '\000') at execQual.c:5206
#22 0x000000000059b9a6 in ExecScan (node=node@entry=0x1e99738, accessMtd=accessMtd@entry=0x5ad780 <PartialSeqNext>,
recheckMtd=recheckMtd@entry=0x5ad770 <PartialSeqRecheck>) at execScan.c:195
#23 0x00000000005ad8d0 in ExecPartialSeqScan (node=node@entry=0x1e99738) at nodePartialSeqscan.c:241
#24 0x0000000000594f68 in ExecProcNode (node=0x1e99738) at execProcnode.c:422
#25 0x00000000005a39b6 in funnel_getnext (funnelstate=0x1e943c8) at nodeFunnel.c:308
#26 ExecFunnel (node=node@entry=0x1e943c8) at nodeFunnel.c:185
#27 0x0000000000594f58 in ExecProcNode (node=0x1e943c8) at execProcnode.c:426
#28 0x00000000005a0212 in ExecAppend (node=node@entry=0x1e941d8) at nodeAppend.c:209
#29 0x0000000000594fa8 in ExecProcNode (node=node@entry=0x1e941d8) at execProcnode.c:399
#30 0x00000000005a0c9e in agg_fill_hash_table (aggstate=0x1e93ba8) at nodeAgg.c:1353
#31 ExecAgg (node=node@entry=0x1e93ba8) at nodeAgg.c:1115
#32 0x0000000000594e38 in ExecProcNode (node=node@entry=0x1e93ba8) at execProcnode.c:506
#33 0x00000000005a8144 in ExecLimit (node=node@entry=0x1e93908) at nodeLimit.c:91
#34 0x0000000000594d98 in ExecProcNode (node=node@entry=0x1e93908) at execProcnode.c:530
#35 0x0000000000592380 in ExecutePlan (dest=0x7f891bbc9f10, direction=<optimized out>, numberTuples=0, sendTuples=1 '\001', operation=CMD_SELECT, planstate=0x1e93908,
#36 standard_ExecutorRun (queryDesc=0x1dbb800, direction=<optimized out>, count=0) at execMain.c:342
#37 0x000000000067e9a8 in PortalRunSelect (portal=0x1e639e0, portal@entry=<error reading variable: Cannot access memory at address 0x7ffff95874c8>,
forward=<optimized out>, count=0, dest=<optimized out>) at pquery.c:947
#0 ScanKeywordLookup (text=text@entry=0x1d57fa0 "information_schema_catalog_name", keywords=0x84f220 <ScanKeywords>, num_keywords=408) at kwlookup.c:64
#1 0x000000000070aa14 in quote_identifier (ident=0x1d57fa0 "information_schema_catalog_name") at ruleutils.c:9009
#2 0x00000000006f54bd in quote_ident (fcinfo=<optimized out>) at quote.c:31
#3 0x0000000000595f72 in ExecMakeFunctionResultNoSets (fcache=0x1d42cb8, econtext=0x1d3f848, isNull=0x1d42858 "", isDone=<optimized out>) at execQual.c:2015
#4 0x0000000000595f1d in ExecMakeFunctionResultNoSets (fcache=0x1d424a8, econtext=0x1d3f848, isNull=0x1d42048 "", isDone=<optimized out>) at execQual.c:1989
#5 0x0000000000595f1d in ExecMakeFunctionResultNoSets (fcache=0x1d41c98, econtext=0x1d3f848, isNull=0x7fff0bdc61df "", isDone=<optimized out>) at execQual.c:1989
#6 0x000000000059b469 in ExecQual (qual=qual@entry=0x1d41368, econtext=econtext@entry=0x1d3f848, resultForNull=resultForNull@entry=0 '\000') at execQual.c:5206
#7 0x000000000059b9a6 in ExecScan (node=node@entry=0x1d3f738, accessMtd=accessMtd@entry=0x5ad780 <PartialSeqNext>,
recheckMtd=recheckMtd@entry=0x5ad770 <PartialSeqRecheck>) at execScan.c:195
#8 0x00000000005ad8d0 in ExecPartialSeqScan (node=node@entry=0x1d3f738) at nodePartialSeqscan.c:241
#9 0x0000000000594f68 in ExecProcNode (node=0x1d3f738) at execProcnode.c:422
#10 0x00000000005a39b6 in funnel_getnext (funnelstate=0x1d3a3c8) at nodeFunnel.c:308
#11 ExecFunnel (node=node@entry=0x1d3a3c8) at nodeFunnel.c:185
#12 0x0000000000594f58 in ExecProcNode (node=0x1d3a3c8) at execProcnode.c:426
#13 0x00000000005a0212 in ExecAppend (node=node@entry=0x1d3a1d8) at nodeAppend.c:209
#14 0x0000000000594fa8 in ExecProcNode (node=node@entry=0x1d3a1d8) at execProcnode.c:399
#15 0x00000000005a0c9e in agg_fill_hash_table (aggstate=0x1d39ba8) at nodeAgg.c:1353
#16 ExecAgg (node=node@entry=0x1d39ba8) at nodeAgg.c:1115
#17 0x0000000000594e38 in ExecProcNode (node=node@entry=0x1d39ba8) at execProcnode.c:506
#18 0x00000000005a8144 in ExecLimit (node=node@entry=0x1d39908) at nodeLimit.c:91
#19 0x0000000000594d98 in ExecProcNode (node=node@entry=0x1d39908) at execProcnode.c:530
#20 0x0000000000592380 in ExecutePlan (dest=0x7fe8c8a1cf10, direction=<optimized out>, numberTuples=0, sendTuples=1 '\001', operation=CMD_SELECT, planstate=0x1d39908,
estate=0x1d01990) at execMain.c:1533
#21 standard_ExecutorRun (queryDesc=0x1c61800, direction=<optimized out>, count=0) at execMain.c:342
#22 0x000000000067e9a8 in PortalRunSelect (portal=portal@entry=0x1d099e0, forward=forward@entry=1 '\001', count=0, count@entry=9223372036854775807,
dest=dest@entry=0x7fe8c8a1cf10) at pquery.c:947
#23 0x000000000067fd0f in PortalRun (portal=portal@entry=0x1d099e0, count=count@entry=9223372036854775807, isTopLevel=isTopLevel@entry=1 '\001',
dest=dest@entry=0x7fe8c8a1cf10, altdest=altdest@entry=0x7fe8c8a1cf10, completionTag=completionTag@entry=0x7fff0bdc6790 "") at pquery.c:791
#24 0x000000000067dab8 in exec_simple_query (
query_string=0x1caf750 "SELECT pg_catalog.quote_ident(c.relname) FROM pg_catalog.pg_class c WHERE c.relkind IN ('r', 'S', 'v', 'm', 'f') AND substring(pg_catalog.quote_ident(c.relname),1,3)='pgb' AND pg_catalog.pg_table_is_v"...) at postgres.c:1107
#25 PostgresMain (argc=<optimized out>, argv=argv@entry=0x1c3db60, dbname=0x1c3da18 "pgbench", username=<optimized out>) at postgres.c:4120
#26 0x0000000000462c8e in BackendRun (port=0x1c621f0) at postmaster.c:4148
#27 BackendStartup (port=0x1c621f0) at postmaster.c:3833
#28 ServerLoop () at postmaster.c:1601
#29 0x000000000062e803 in PostmasterMain (argc=argc@entry=1, argv=argv@entry=0x1c3cca0) at postmaster.c:1248
#30 0x00000000004636dd in main (argc=1, argv=0x1c3cca0) at main.c:221
#0 0x000000000075d757 in hash_search_with_hash_value (hashp=0x1d62310, keyPtr=keyPtr@entry=0x7fffb686f4a0, hashvalue=hashvalue@entry=171639189,
action=action@entry=HASH_ENTER, foundPtr=foundPtr@entry=0x7fffb686f44f <Address 0x7fffb686f44f out of bounds>) at dynahash.c:1026
#1 0x0000000000654d52 in BufTableInsert (tagPtr=tagPtr@entry=0x7fffb686f4a0, hashcode=hashcode@entry=171639189, buf_id=169) at buf_table.c:128
#2 0x0000000000657711 in BufferAlloc (foundPtr=0x7fffb686f49f <Address 0x7fffb686f49f out of bounds>, strategy=0x0, blockNum=11, forkNum=MAIN_FORKNUM,
relpersistence=<error reading variable: Cannot access memory at address 0x7fffb686f484>,
smgr=<error reading variable: Cannot access memory at address 0x7fffb686f488>) at bufmgr.c:1089
#3 ReadBuffer_common (smgr=<error reading variable: Cannot access memory at address 0x7fffb686f488>, relpersistence=<optimized out>, forkNum=MAIN_FORKNUM,
forkNum@entry=<error reading variable: Cannot access memory at address 0x7fffb686f4f0>, blockNum=11,
blockNum@entry=<error reading variable: Cannot access memory at address 0x7fffb686f4f8>, mode=RBM_NORMAL, strategy=0x0,
hit=hit@entry=0x7fffb686f54f <Address 0x7fffb686f54f out of bounds>) at bufmgr.c:641
#4 0x0000000000657e40 in ReadBufferExtended (reln=<error reading variable: Cannot access memory at address 0x7fffb686f4e8>,
reln@entry=<error reading variable: Cannot access memory at address 0x7fffb686f568>,
forkNum=<error reading variable: Cannot access memory at address 0x7fffb686f4f0>,
blockNum=<error reading variable: Cannot access memory at address 0x7fffb686f4f8>, mode=<optimized out>, strategy=<optimized out>) at bufmgr.c:560
>
> On 25 March 2015 at 10:27, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> Fixed the reported issue on assess-parallel-safety thread and another
>> bug caught while testing joins and integrated with latest version of
>> parallel-mode patch (parallel-mode-v9 patch).
>>
>> Apart from that I have moved the Initialization of dsm segement from
>> InitNode phase to ExecFunnel() (on first execution) as per suggestion
>> from Robert. The main idea is that as it creates large shared memory
>> segment, so do the work when it is really required.
>>
>>
>> HEAD Commit-Id: 11226e38
>> parallel-mode-v9.patch [2]
>> assess-parallel-safety-v4.patch [1]
>> parallel-heap-scan.patch [3]
>> parallel_seqscan_v12.patch (Attached with this mail)
>>
>> [1] - http://www.postgresql.org/message-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
>> [2] - http://www.postgresql.org/message-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
>> [3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
>
>
> Okay, with my pgbench_accounts partitioned into 300, I ran:
>
> SELECT DISTINCT bid FROM pgbench_accounts;
>
> The query never returns,
>
> grep -r 'starting background worker process "parallel worker for PID 12165"' postgresql-2015-03-25_112522.log | wc -l
> 2496
>
> 2,496 workers? This is with parallel_seqscan_degree set to 8. If I set it to 2, this number goes down to 626, and with 16, goes up to 4320.
>
> Still not sure why 8 workers are needed for each partial scan. I would expect 8 workers to be used for 8 separate scans. Perhaps this is just my misunderstanding of how this feature works.
>
On Wed, Mar 25, 2015 at 5:16 PM, Thom Brown <thom@linux.com> wrote:
>
> On 25 March 2015 at 10:27, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> Fixed the reported issue on assess-parallel-safety thread and another
>> bug caught while testing joins and integrated with latest version of
>> parallel-mode patch (parallel-mode-v9 patch).
>>
>> Apart from that I have moved the Initialization of dsm segement from
>> InitNode phase to ExecFunnel() (on first execution) as per suggestion
>> from Robert. The main idea is that as it creates large shared memory
>> segment, so do the work when it is really required.
>>
>>
>> HEAD Commit-Id: 11226e38
>> parallel-mode-v9.patch [2]
>> assess-parallel-safety-v4.patch [1]
>> parallel-heap-scan.patch [3]
>> parallel_seqscan_v12.patch (Attached with this mail)
>>
>> [1] - http://www.postgresql.org/message-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
>> [2] - http://www.postgresql.org/message-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
>> [3] - http://www.postgresql.org/message-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
>
>
> Okay, with my pgbench_accounts partitioned into 300, I ran:
>
> SELECT DISTINCT bid FROM pgbench_accounts;
>
> The query never returns,You seem to be hitting the issue I have pointed in near-by thread [1]and I have mentioned the same while replying on assess-parallel-safetythread. Can you check after applying the patch in mail [1]
parallel-mode-v9.patch
assess-parallel-safety-v4.patch
parallel-heap-scan.patch
parallel_seqscan_v12.patch
release_lock_dsm_v1.patch
> and I also get this:
>
> grep -r 'starting background worker process "parallel worker for PID 12165"' postgresql-2015-03-25_112522.log | wc -l
> 2496
>
> 2,496 workers? This is with parallel_seqscan_degree set to 8. If I set it to 2, this number goes down to 626, and with 16, goes up to 4320.
>..>
> Still not sure why 8 workers are needed for each partial scan. I would expect 8 workers to be used for 8 separate scans. Perhaps this is just my misunderstanding of how this feature works.
>The reason is that for each table scan, it tries to use workersequal to parallel_seqscan_degree if they are available and in thiscase as the scan for inheritance hierarchy (tables in hierarchy) happensone after another, it uses 8 workers for each scan. I think as of nowthe strategy to decide number of workers to be used in scan is keptsimple and in future we can try to come with some better mechanismto decide number of workers.
--
>
> On 25 March 2015 at 15:49, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> On Wed, Mar 25, 2015 at 5:16 PM, Thom Brown <thom@linux.com> wrote:
>> > Okay, with my pgbench_accounts partitioned into 300, I ran:
>> >
>> > SELECT DISTINCT bid FROM pgbench_accounts;
>> >
>> > The query never returns,
>>
>> You seem to be hitting the issue I have pointed in near-by thread [1]
>> and I have mentioned the same while replying on assess-parallel-safety
>> thread. Can you check after applying the patch in mail [1]
>
>
> Ah, okay, here's the patches I've now applied:
>
> parallel-mode-v9.patch
> assess-parallel-safety-v4.patch
> parallel-heap-scan.patch
> parallel_seqscan_v12.patch
> release_lock_dsm_v1.patch
>
> (with perl patch for pg_proc.h)
>
> The query now returns successfully.
>
>> >
>> > Still not sure why 8 workers are needed for each partial scan. I would expect 8 workers to be used for 8 separate scans. Perhaps this is just my misunderstanding of how this feature works.
>> >
>>
>> The reason is that for each table scan, it tries to use workers
>> equal to parallel_seqscan_degree if they are available and in this
>> case as the scan for inheritance hierarchy (tables in hierarchy) happens
>> one after another, it uses 8 workers for each scan. I think as of now
>> the strategy to decide number of workers to be used in scan is kept
>> simple and in future we can try to come with some better mechanism
>> to decide number of workers.
>
>
> Yes, I was expecting the parallel aspect to apply across partitions (a worker per partition up to parallel_seqscan_degree and reallocate to another >scan once finished with current job), not individual ones,
parallel_setup_cost, parallel_startup_cost) for costing of parallel plans, so
>
> On 25 March 2015 at 11:46, Thom Brown <thom@linux.com> wrote:
>>
>>
>> Still not sure why 8 workers are needed for each partial scan. I would expect 8 workers to be used for 8 separate scans. Perhaps this is just my misunderstanding of how this feature works.
>
>
> Another issue:
>
> SELECT * FROM pgb<tab>
>
> *crash*
>
The reason of this problem is that above tab-completion is executing
Attachment
On Fri, Mar 27, 2015 at 2:34 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > The reason of this problem is that above tab-completion is executing > query [1] which contains subplan for the funnel node and currently > we don't have capability (enough infrastructure) to support execution > of subplans by parallel workers. Here one might wonder why we > have choosen Parallel Plan (Funnel node) for such a case and the > reason for same is that subplans are attached after Plan generation > (SS_finalize_plan()) and if want to discard such a plan, it will be > much more costly, tedious and not worth the effort as we have to > eventually make such a plan work. > > Here we have two choices to proceed, first one is to support execution > of subplans by parallel workers and second is execute/scan locally for > Funnel node having subplan (don't launch workers). It looks to me like the is an InitPlan, not a subplan. There shouldn't be any problem with a Funnel node having an InitPlan; it looks to me like all of the InitPlan stuff is handled by common code within the executor (grep for initPlan), so it ought to work here the same as it does for anything else. What I suspect is failing (although you aren't being very clear about it here) is the passing down of the parameters set by the InitPlan to the workers. I think we need to make that work; it's an integral piece of the executor infrastructure and we shouldn't leave it out just because it requires a bit more IPC. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Mar 18, 2015 at 11:43 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I think I figured out the problem. That fix only helps in the case >> where the postmaster noticed the new registration previously but >> didn't start the worker, and then later notices the termination. >> What's much more likely to happen is that the worker is started and >> terminated so quickly that both happen before we create a >> RegisteredBgWorker for it. The attached patch fixes that case, too. > > Patch fixes the problem and now for Rescan, we don't need to Wait > for workers to finish. I realized that there is a problem with this. If an error occurs in one of the workers just as we're deciding to kill them all, then the error won't be reported. Also, the new code to propagate XactLastRecEnd won't work right, either. I think we need to find a way to shut down the workers cleanly. The idea generally speaking should be: 1. Tell all of the workers that we want them to shut down gracefully without finishing the scan. 2. Wait for them to exit via WaitForParallelWorkersToFinish(). My first idea about how to implement this is to have the master detach all of the tuple queues via a new function TupleQueueFunnelShutdown(). Then, we should change tqueueReceiveSlot() so that it does not throw an error when shm_mq_send() returns SHM_MQ_DETACHED. We could modify the receiveSlot method of a DestReceiver to return bool rather than void; a "true" value can mean "continue processing" where as a "false" value can mean "stop early, just as if we'd reached the end of the scan". This design will cause each parallel worker to finish producing the tuple it's currently in the middle of generating, and then shut down. You can imagine cases where we'd want the worker to respond faster than that, though; for example, if it's applying a highly selective filter condition, we'd like it to stop the scan right away, not when it finds the next matching tuple. I can't immediately see a real clean way of accomplishing that, though. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Mar 25, 2015 at 6:27 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Apart from that I have moved the Initialization of dsm segement from > InitNode phase to ExecFunnel() (on first execution) as per suggestion > from Robert. The main idea is that as it creates large shared memory > segment, so do the work when it is really required. So, suppose we have a plan like this: Append -> Funnel -> Partial Seq Scan -> Funnel -> Partial Seq Scan (repeated many times) In earlier versions of this patch, that was chewing up lots of DSM segments. But it seems to me, on further reflection, that it should never use more than one at a time. The first funnel node should initialize its workers and then when it finishes, all those workers should get shut down cleanly and the DSM destroyed before the next scan is initialized. Obviously we could do better here: if we put the Funnel on top of the Append instead of underneath it, we could avoid shutting down and restarting workers for every child node. But even without that, I'm hoping it's no longer the case that this uses more than one DSM at a time. If that's not the case, we should see if we can't fix that. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Mar 27, 2015 at 2:34 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > The reason of this problem is that above tab-completion is executing
> > query [1] which contains subplan for the funnel node and currently
> > we don't have capability (enough infrastructure) to support execution
> > of subplans by parallel workers. Here one might wonder why we
> > have choosen Parallel Plan (Funnel node) for such a case and the
> > reason for same is that subplans are attached after Plan generation
> > (SS_finalize_plan()) and if want to discard such a plan, it will be
> > much more costly, tedious and not worth the effort as we have to
> > eventually make such a plan work.
> >
> > Here we have two choices to proceed, first one is to support execution
> > of subplans by parallel workers and second is execute/scan locally for
> > Funnel node having subplan (don't launch workers).
>
> It looks to me like the is an InitPlan, not a subplan. There
> shouldn't be any problem with a Funnel node having an InitPlan; it
> looks to me like all of the InitPlan stuff is handled by common code
> within the executor (grep for initPlan), so it ought to work here the
> same as it does for anything else. What I suspect is failing
> (although you aren't being very clear about it here) is the passing
> down of the parameters set by the InitPlan to the workers.
>
> On Wed, Mar 25, 2015 at 6:27 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Apart from that I have moved the Initialization of dsm segement from
> > InitNode phase to ExecFunnel() (on first execution) as per suggestion
> > from Robert. The main idea is that as it creates large shared memory
> > segment, so do the work when it is really required.
>
> So, suppose we have a plan like this:
>
> Append
> -> Funnel
> -> Partial Seq Scan
> -> Funnel
> -> Partial Seq Scan
> (repeated many times)
>
> In earlier versions of this patch, that was chewing up lots of DSM
> segments. But it seems to me, on further reflection, that it should
> never use more than one at a time. The first funnel node should
> initialize its workers and then when it finishes, all those workers
> should get shut down cleanly and the DSM destroyed before the next
> scan is initialized.
>
> Obviously we could do better here: if we put the Funnel on top of the
> Append instead of underneath it, we could avoid shutting down and
> restarting workers for every child node. But even without that, I'm
> hoping it's no longer the case that this uses more than one DSM at a
> time. If that's not the case, we should see if we can't fix that.
>
Currently it doesn't behave you are expecting, it destroys the DSM and
>
> On Wed, Mar 18, 2015 at 11:43 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> I think I figured out the problem. That fix only helps in the case
> >> where the postmaster noticed the new registration previously but
> >> didn't start the worker, and then later notices the termination.
> >> What's much more likely to happen is that the worker is started and
> >> terminated so quickly that both happen before we create a
> >> RegisteredBgWorker for it. The attached patch fixes that case, too.
> >
> > Patch fixes the problem and now for Rescan, we don't need to Wait
> > for workers to finish.
>
> I realized that there is a problem with this. If an error occurs in
> one of the workers just as we're deciding to kill them all, then the
> error won't be reported.
> XactLastRecEnd won't work right, either.
+ latestXid = RecordTransactionAbort(false);
+ else
+ {
+ latestXid = InvalidTransactionId;
+
+ /*
+ * Since the parallel master won't get our value of XactLastRecEnd in this
+ * case, we nudge WAL-writer ourselves in this case. See related comments in
+ * RecordTransactionAbort for why this matters.
+ */
+ XLogSetAsyncXactLSN(XactLastRecEnd);
+ }
On Tue, Mar 31, 2015 at 8:53 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> It looks to me like the is an InitPlan, not a subplan. There >> shouldn't be any problem with a Funnel node having an InitPlan; it >> looks to me like all of the InitPlan stuff is handled by common code >> within the executor (grep for initPlan), so it ought to work here the >> same as it does for anything else. What I suspect is failing >> (although you aren't being very clear about it here) is the passing >> down of the parameters set by the InitPlan to the workers. > > It is failing because we are not passing InitPlan itself (InitPlan is > nothing but a list of SubPlan) and I tried tried to describe in previous > mail [1] what we need to do to achieve the same, but in short, it is not > difficult to pass down the required parameters (like plan->InitPlan or > plannedstmt->subplans), rather the main missing part is the handling > of such parameters in worker side (mainly we need to provide support > for all plan nodes which can be passed as part of InitPlan in readfuncs.c). > I am not against supporting InitPlan's on worker side, but just wanted to > say that if possible why not leave that for first version. Well, if we *don't* handle it, we're going to need to insert some hack to ensure that the planner doesn't create plans. And that seems pretty unappealing. Maybe it'll significantly compromise plan quality, and maybe it won't, but at the least, it's ugly. > [1] > I have tried to evaluate what it would take us to support execution > of subplans by parallel workers. We need to pass the sub plans > stored in Funnel Node (initPlan) and corresponding subplans stored > in planned statement (subplans) as subplan's stored in Funnel node > has reference to subplans in planned statement. Next currently > readfuncs.c (functions to read different type of nodes) doesn't support > reading any type of plan node, so we need to add support for reading all > kind > of plan nodes (as subplan can have any type of plan node) and similarly > to execute any type of Plan node, we might need more work (infrastructure). I don't think you need to do anything that complicated. I'm not proposing to *run* the initPlan in the workers, just to pass the parameter values down. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Apr 1, 2015 at 6:30 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Mon, Mar 30, 2015 at 8:35 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> So, suppose we have a plan like this: >> >> Append >> -> Funnel >> -> Partial Seq Scan >> -> Funnel >> -> Partial Seq Scan >> (repeated many times) >> >> In earlier versions of this patch, that was chewing up lots of DSM >> segments. But it seems to me, on further reflection, that it should >> never use more than one at a time. The first funnel node should >> initialize its workers and then when it finishes, all those workers >> should get shut down cleanly and the DSM destroyed before the next >> scan is initialized. >> >> Obviously we could do better here: if we put the Funnel on top of the >> Append instead of underneath it, we could avoid shutting down and >> restarting workers for every child node. But even without that, I'm >> hoping it's no longer the case that this uses more than one DSM at a >> time. If that's not the case, we should see if we can't fix that. >> > Currently it doesn't behave you are expecting, it destroys the DSM and > perform clean shutdown of workers (DestroyParallelContext()) at the > time of ExecEndFunnel() which in this case happens when we finish > Execution of AppendNode. > > One way to change it is do the clean up for parallel context when we > fetch last tuple from the FunnelNode (into ExecFunnel) as at that point > we are sure that we don't need workers or dsm anymore. Does that > sound reasonable to you? Yeah, I think that's exactly what we should do. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Apr 1, 2015 at 7:30 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> > Patch fixes the problem and now for Rescan, we don't need to Wait >> > for workers to finish. >> >> I realized that there is a problem with this. If an error occurs in >> one of the workers just as we're deciding to kill them all, then the >> error won't be reported. > > We are sending SIGTERM to worker for terminating the worker, so > if the error occurs before the signal is received then it should be > sent to master backend. Am I missing something here? The master only checks for messages at intervals - each CHECK_FOR_INTERRUPTS(), basically. So when the master terminates the workers, any errors generated after the last check for messages will be lost. >> Also, the new code to propagate >> XactLastRecEnd won't work right, either. > > As we are generating FATAL error on termination of worker > (bgworker_die()), so won't it be handled in AbortTransaction path > by below code in parallel-mode patch? That will asynchronously flush the WAL, but if the master goes on to commit, we've wait synchronously for WAL flush, and possibly sync rep. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Tue, Mar 31, 2015 at 8:53 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> It looks to me like the is an InitPlan, not a subplan. There
> >> shouldn't be any problem with a Funnel node having an InitPlan; it
> >> looks to me like all of the InitPlan stuff is handled by common code
> >> within the executor (grep for initPlan), so it ought to work here the
> >> same as it does for anything else. What I suspect is failing
> >> (although you aren't being very clear about it here) is the passing
> >> down of the parameters set by the InitPlan to the workers.
> >
> > It is failing because we are not passing InitPlan itself (InitPlan is
> > nothing but a list of SubPlan) and I tried tried to describe in previous
> > mail [1] what we need to do to achieve the same, but in short, it is not
> > difficult to pass down the required parameters (like plan->InitPlan or
> > plannedstmt->subplans), rather the main missing part is the handling
> > of such parameters in worker side (mainly we need to provide support
> > for all plan nodes which can be passed as part of InitPlan in readfuncs.c).
> > I am not against supporting InitPlan's on worker side, but just wanted to
> > say that if possible why not leave that for first version.
>
> Well, if we *don't* handle it, we're going to need to insert some hack
> to ensure that the planner doesn't create plans. And that seems
> pretty unappealing. Maybe it'll significantly compromise plan
> quality, and maybe it won't, but at the least, it's ugly.
>
> > [1]
> > I have tried to evaluate what it would take us to support execution
> > of subplans by parallel workers. We need to pass the sub plans
> > stored in Funnel Node (initPlan) and corresponding subplans stored
> > in planned statement (subplans) as subplan's stored in Funnel node
> > has reference to subplans in planned statement. Next currently
> > readfuncs.c (functions to read different type of nodes) doesn't support
> > reading any type of plan node, so we need to add support for reading all
> > kind
> > of plan nodes (as subplan can have any type of plan node) and similarly
> > to execute any type of Plan node, we might need more work (infrastructure).
>
> I don't think you need to do anything that complicated. I'm not
> proposing to *run* the initPlan in the workers, just to pass the
> parameter values down.
>
Sorry, but I am not able to understand how it will help if just parameters
Bitmapset *allParam; in Plan structure) are passed to workers and I
On Wed, Apr 1, 2015 at 10:28 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Well, if we *don't* handle it, we're going to need to insert some hack >> to ensure that the planner doesn't create plans. And that seems >> pretty unappealing. Maybe it'll significantly compromise plan >> quality, and maybe it won't, but at the least, it's ugly. > > I also think changing anything in planner related to this is not a > good idea, but what about detecting this during execution (into > ExecFunnel) and then just run the plan locally (by master backend)? That seems like an even bigger hack; we want to minimize the number of cases where we create a parallel plan and then don't go parallel. Doing that in the hopefully-rare case where we manage to blow out the DSM segments seems OK, but doing it every time a plan of a certain type gets created doesn't seem very appealing to me. >> > [1] >> > I have tried to evaluate what it would take us to support execution >> > of subplans by parallel workers. We need to pass the sub plans >> > stored in Funnel Node (initPlan) and corresponding subplans stored >> > in planned statement (subplans) as subplan's stored in Funnel node >> > has reference to subplans in planned statement. Next currently >> > readfuncs.c (functions to read different type of nodes) doesn't support >> > reading any type of plan node, so we need to add support for reading all >> > kind >> > of plan nodes (as subplan can have any type of plan node) and similarly >> > to execute any type of Plan node, we might need more work >> > (infrastructure). >> >> I don't think you need to do anything that complicated. I'm not >> proposing to *run* the initPlan in the workers, just to pass the >> parameter values down. > > Sorry, but I am not able to understand how it will help if just parameters > (If I understand correctly you want to say about Bitmapset *extParam; > Bitmapset *allParam; in Plan structure) are passed to workers and I > think they are already getting passed only initPlan and related Subplan > in planned statement is not passed and the reason is that ss_finalize_plan() > attaches initPlan to top node (which in this case is Funnel node and not > PartialSeqScan) > > By any chance, do you mean that we run the part of the statement in > workers and then run initPlan in master backend? If I'm not confused, it would be the other way around. We would run the initPlan in the master backend *first* and then the rest in the workers. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> >> I don't think you need to do anything that complicated. I'm not
> >> proposing to *run* the initPlan in the workers, just to pass the
> >> parameter values down.
> >
> > Sorry, but I am not able to understand how it will help if just parameters
> > (If I understand correctly you want to say about Bitmapset *extParam;
> > Bitmapset *allParam; in Plan structure) are passed to workers and I
> > think they are already getting passed only initPlan and related Subplan
> > in planned statement is not passed and the reason is that ss_finalize_plan()
> > attaches initPlan to top node (which in this case is Funnel node and not
> > PartialSeqScan)
> >
> > By any chance, do you mean that we run the part of the statement in
> > workers and then run initPlan in master backend?
>
> If I'm not confused, it would be the other way around. We would run
> the initPlan in the master backend *first* and then the rest in the
> workers.
>
>
> On Wed, Apr 1, 2015 at 7:30 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> Also, the new code to propagate
> >> XactLastRecEnd won't work right, either.
> >
> > As we are generating FATAL error on termination of worker
> > (bgworker_die()), so won't it be handled in AbortTransaction path
> > by below code in parallel-mode patch?
>
> That will asynchronously flush the WAL, but if the master goes on to
> commit, we've wait synchronously for WAL flush, and possibly sync rep.
>
Okay, so you mean if master backend later commits, then there is
On Thu, Apr 2, 2015 at 2:36 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> If I'm not confused, it would be the other way around. We would run >> the initPlan in the master backend *first* and then the rest in the >> workers. > > Either one of us is confused, let me try to describe my understanding in > somewhat more detail. Let me try to explain w.r.t the tab completion > query [1]. In this, the initPlan is generated for Qualification expression > [2], so it will be executed during qualification and the callstack will > look like: > > postgres.exe!ExecSeqScan(ScanState * node=0x000000000c33bce8) Line 113 C > postgres.exe!ExecProcNode(PlanState * node=0x000000000c33bce8) Line 418 + > 0xa bytes C > postgres.exe!ExecSetParamPlan(SubPlanState * node=0x000000000c343930, > ExprContext * econtext=0x000000000c33de50) Line 1001 + 0xa bytes C >> postgres.exe!ExecEvalParamExec(ExprState * exprstate=0x000000000c33f980, >> ExprContext * econtext=0x000000000c33de50, char * isNull=0x000000000c33f481, >> ExprDoneCond * isDone=0x0000000000000000) Line 1111 C > postgres.exe!ExecMakeFunctionResultNoSets(FuncExprState * > fcache=0x000000000c33f0d0, ExprContext * econtext=0x000000000c33de50, char * > isNull=0x000000000042f1c8, ExprDoneCond * isDone=0x0000000000000000) Line > 1992 + 0x2d bytes C > postgres.exe!ExecEvalOper(FuncExprState * fcache=0x000000000c33f0d0, > ExprContext * econtext=0x000000000c33de50, char * isNull=0x000000000042f1c8, > ExprDoneCond * isDone=0x0000000000000000) Line 2443 C > postgres.exe!ExecQual(List * qual=0x000000000c33fa08, ExprContext * > econtext=0x000000000c33de50, char resultForNull=0) Line 5206 + 0x1a bytes C > postgres.exe!ExecScan(ScanState * node=0x000000000c33dd38, TupleTableSlot > * (ScanState *)* accessMtd=0x0000000140232940, char (ScanState *, > TupleTableSlot *)* recheckMtd=0x00000001402329e0) Line 195 + 0x1a bytes C > postgres.exe!ExecSeqScan(ScanState * node=0x000000000c33dd38) Line 114 C > > Basically here initPlan is getting executed during Qualification. OK, I failed to realize that the initPlan doesn't get evaluated until first use. Maybe in the case of a funnel node we should force all of the initplans to be run before starting parallelism, so that we can pass down the resulting value to each worker. If we try to push the whole plan tree down from the worker then, aside from the issue of needing to copy the plan tree, it'll get evaluated N times instead of once. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Apr 2, 2015 at 3:07 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Apr 1, 2015 at 6:11 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> On Wed, Apr 1, 2015 at 7:30 AM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> >> Also, the new code to propagate >> >> XactLastRecEnd won't work right, either. >> > >> > As we are generating FATAL error on termination of worker >> > (bgworker_die()), so won't it be handled in AbortTransaction path >> > by below code in parallel-mode patch? >> >> That will asynchronously flush the WAL, but if the master goes on to >> commit, we've wait synchronously for WAL flush, and possibly sync rep. > > Okay, so you mean if master backend later commits, then there is > a chance of loss of WAL data written by worker. > Can't we report the location to master as the patch does in case of > Commit in worker? That's exactly why I think it needs to call WaitForParallelWorkersToFinish() - because it will do just that. We only need to invent a way of telling the worker to stop the scan and shut down cleanly. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
I think we're getting to the point where having a unique mapping from
the plan to the execution tree is proving to be rather limiting
anyway. Check for example discussion about join removal. But even for
current code, showing only the custom plans for the first five EXPLAIN
EXECUTEs is pretty nasty (Try explain that to somebody that doesn't know
pg internals. Their looks are worth gold and can kill you at the same
time) and should be done differently.
David Rowley <dgrowleyml@gmail.com> wrote: > If we attempt to do this parallel stuff at plan time, and we > happen to plan at some quiet period, or perhaps worse, some > application's start-up process happens to PREPARE a load of > queries when the database is nice and quite, then quite possibly > we'll end up with some highly parallel queries. Then perhaps come > the time these queries are actually executed the server is very > busy... Things will fall apart quite quickly due to the masses of > IPC and context switches that would be going on. > > I completely understand that this parallel query stuff is all > quite new to us all and we're likely still trying to nail down > the correct infrastructure for it to work well, so this is why > I'm proposing that the planner should know nothing of parallel > query, instead I think it should work more along the lines of: > > * Planner should be completely oblivious to what parallel query > is. > * Before executor startup the plan is passed to a function which > decides if we should parallelise it, and does so if the plan > meets the correct requirements. This should likely have a very > fast exit path such as: > if root node's cost < parallel_query_cost_threshold > return; /* the query is not expensive enough to attempt to make parallel */ > > The above check will allow us to have an almost zero overhead for > small low cost queries. > > This function would likely also have some sort of logic in order > to determine if the server has enough spare resource at the > current point in time to allow queries to be parallelised There is a lot to like about this suggestion. I've seen enough performance crashes due to too many concurrent processes (even when each connection can only use a single process) to believe that, for a plan which will be saved, it is possible to know at planning time whether parallelization will be a nice win or a devastating over-saturation of resources during some later execution phase. Another thing to consider is that this is not entirely unrelated to the concept of admission control policies. Perhaps this phase could be a more general execution start-up admission control phase, where parallel processing would be one adjustment that could be considered. Initially it might be the *only* consideration, but it might be good to try to frame it in a way that allowed implementation of other policies, too. -- Kevin Grittner EDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Apr 4, 2015 at 5:19 AM, David Rowley <dgrowleyml@gmail.com> wrote: > Going over the previous emails in this thread I see that it has been a long > time since anyone discussed anything around how we might decide at planning > time how many workers should be used for the query, and from the emails I > don't recall anyone proposing a good idea about how this might be done, and > I for one can't see how this is at all possible to do at planning time. > > I think that the planner should know nothing of parallel query at all, and > the planner quite possibly should go completely unmodified for this patch. > One major problem I can see is that, given a query such as: > > SELECT * FROM million_row_product_table WHERE category = 'ELECTRONICS'; > > Where we have a non-unique index on category, some plans which may be > considered might be: > > 1. Index scan on the category index to get all rows matching 'ELECTRONICS' > 2. Sequence scan on the table, filter matching rows. > 3. Parallel plan which performs a series of partial sequence scans pulling > out all matching rows. > > I really think that if we end choosing things like plan 3, when plan 2 was > thrown out because of its cost, then we'll end up consuming more CPU and I/O > than we can possibly justify using. The environmentalist in me screams that > this is wrong. What if we kicked off 128 worker process on some high-end > hardware to do this? I certainly wouldn't want to pay the power bill. I > understand there's costing built in to perhaps stop this, but I still think > it's wrong headed, and we need to still choose the fastest non-parallel plan > and only consider parallelising that later. I agree that this is an area that needs more thought. I don't (currently, anyway) agree that the planner shouldn't know anything about parallelism. The problem with that is that there's lots of relevant stuff that can only be known at plan time. For example, consider the query you mention above on a table with no index. If the WHERE clause is highly selective, a parallel plan may well be best. But if the selectivity is only, say, 50%, a parallel plan is stupid: the IPC costs of shipping many rows back to the master will overwhelm any benefit we could possibly have hoped to get, and the overall result will likely be that the parallel plan both runs slower and uses more resources. At plan time, we have the selectivity information conveniently at hand, and can use that as part of the cost model to make educated decisions. Execution time is way too late to be thinking about those kinds of questions. I think one of the philosophical questions that has to be answered here is "what does it mean to talk about the cost of a parallel plan?". For a non-parallel plan, the cost of the plan means both "the amount of effort we will spend executing the plan" and also "the amount of time we think the plan will take to complete", but those two things are different for parallel plans. I'm inclined to think it's right to view the cost of a parallel plan as a proxy for execution time, because the fundamental principle of the planner is that we pick the lowest-cost plan. But there also clearly needs to be some way to prevent the selection of a plan which runs slightly faster at the cost of using vastly more resources. Currently, the planner tracks the best unsorted path for each relation as well as the best path for each useful sort order. Suppose we treat parallelism as another axis for judging the quality of a plan: we keep the best unsorted, non-parallel path; the best non-parallel path for each useful sort order; the best unsorted, parallel path; and the best parallel path for each sort order. Each time we plan a node, we generate non-parallel paths first, and then parallel paths. But, if a parallel plan isn't markedly faster than the non-parallel plan for the same sort order, then we discard it. I'm not sure exactly what the thresholds should be here, and they probably need to be configurable, because on a single-user system with excess capacity available it may be absolutely desirable to use ten times the resources to get an answer 25% faster, but on a heavy-loaded system that will stink. Some ideas for GUCs: max_parallel_degree = The largest number of processes we'll consider using for a single query. min_parallel_speedup = The minimum percentage by which a parallel path must be cheaper (in terms of execution time) than a non-parallel path in order to survive. I'm imagining the default here might be something like 15%. min_parallel_speedup_per_worker = Like the previous one, but per worker. e.g. if this is 5%, which might be a sensible default, then a plan with 4 workers must be at least 20% better to survive, but a plan using only 2 workers only needs to be 10% better. An additional benefit of this line of thinking is that planning would always produce a best non-parallel path. And sometimes, there would also be a best parallel path that is expected to run faster. We could then choose between them dynamically at execution time. I think it's pretty hard to imagine a scenario as extreme as the one you mention above ever actually occurring in practice. I mean, even the most naive implementation of parallel query will presumably have something like max_parallel_degree, and you probably won't have that set to 128. For starters, it can't possibly make sense unless you server has at least 128 CPUs, and even then it only makes sense if you don't mind a single query using all of them, and even if the first of those things is true, the second one probably isn't. I don't doubt that less extreme variants of this scenario are possible, though. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> David Rowley <dgrowleyml@gmail.com> wrote:
>
> > If we attempt to do this parallel stuff at plan time, and we
> > happen to plan at some quiet period, or perhaps worse, some
> > application's start-up process happens to PREPARE a load of
> > queries when the database is nice and quite, then quite possibly
> > we'll end up with some highly parallel queries. Then perhaps come
> > the time these queries are actually executed the server is very
> > busy... Things will fall apart quite quickly due to the masses of
> > IPC and context switches that would be going on.
> >
> > I completely understand that this parallel query stuff is all
> > quite new to us all and we're likely still trying to nail down
> > the correct infrastructure for it to work well, so this is why
> > I'm proposing that the planner should know nothing of parallel
> > query, instead I think it should work more along the lines of:
> >
> > * Planner should be completely oblivious to what parallel query
> > is.
> > * Before executor startup the plan is passed to a function which
> > decides if we should parallelise it, and does so if the plan
> > meets the correct requirements. This should likely have a very
> > fast exit path such as:
> > if root node's cost < parallel_query_cost_threshold
> > return; /* the query is not expensive enough to attempt to make parallel */
> >
> > The above check will allow us to have an almost zero overhead for
> > small low cost queries.
> >
> > This function would likely also have some sort of logic in order
> > to determine if the server has enough spare resource at the
> > current point in time to allow queries to be parallelised
>
> There is a lot to like about this suggestion.
>
> I've seen enough performance crashes due to too many concurrent
> processes (even when each connection can only use a single process)
> to believe that, for a plan which will be saved, it is possible to
> know at planning time whether parallelization will be a nice win or
> a devastating over-saturation of resources during some later
> execution phase.
>
> Another thing to consider is that this is not entirely unrelated to
> the concept of admission control policies. Perhaps this phase
> could be a more general execution start-up admission control phase,
> where parallel processing would be one adjustment that could be
> considered.
>
> I agree that this is an area that needs more thought. I don't
> (currently, anyway) agree that the planner shouldn't know anything
> about parallelism. The problem with that is that there's lots of
> relevant stuff that can only be known at plan time. For example,
> consider the query you mention above on a table with no index. If the
> WHERE clause is highly selective, a parallel plan may well be best.
> But if the selectivity is only, say, 50%, a parallel plan is stupid:
> the IPC costs of shipping many rows back to the master will overwhelm
> any benefit we could possibly have hoped to get, and the overall
> result will likely be that the parallel plan both runs slower and uses
> more resources. At plan time, we have the selectivity information
> conveniently at hand, and can use that as part of the cost model to
> make educated decisions. Execution time is way too late to be
> thinking about those kinds of questions.
>
> I think one of the philosophical questions that has to be answered
> here is "what does it mean to talk about the cost of a parallel
> plan?". For a non-parallel plan, the cost of the plan means both "the
> amount of effort we will spend executing the plan" and also "the
> amount of time we think the plan will take to complete", but those two
> things are different for parallel plans. I'm inclined to think it's
> right to view the cost of a parallel plan as a proxy for execution
> time, because the fundamental principle of the planner is that we pick
> the lowest-cost plan. But there also clearly needs to be some way to
> prevent the selection of a plan which runs slightly faster at the cost
> of using vastly more resources.
>
> Currently, the planner tracks the best unsorted path for each relation
> as well as the best path for each useful sort order. Suppose we treat
> parallelism as another axis for judging the quality of a plan: we keep
> the best unsorted, non-parallel path; the best non-parallel path for
> each useful sort order; the best unsorted, parallel path; and the best
> parallel path for each sort order. Each time we plan a node, we
> generate non-parallel paths first, and then parallel paths. But, if a
> parallel plan isn't markedly faster than the non-parallel plan for the
> same sort order, then we discard it.
I think one of the philosophical questions that has to be answered
here is "what does it mean to talk about the cost of a parallel
plan?". For a non-parallel plan, the cost of the plan means both "the
amount of effort we will spend executing the plan" and also "the
amount of time we think the plan will take to complete", but those two
things are different for parallel plans. I'm inclined to think it's
right to view the cost of a parallel plan as a proxy for execution
time, because the fundamental principle of the planner is that we pick
the lowest-cost plan. But there also clearly needs to be some way to
prevent the selection of a plan which runs slightly faster at the cost
of using vastly more resources.
Currently, the planner tracks the best unsorted path for each relation
as well as the best path for each useful sort order. Suppose we treat
parallelism as another axis for judging the quality of a plan: we keep
the best unsorted, non-parallel path; the best non-parallel path for
each useful sort order; the best unsorted, parallel path; and the best
parallel path for each sort order. Each time we plan a node, we
generate non-parallel paths first, and then parallel paths. But, if a
parallel plan isn't markedly faster than the non-parallel plan for the
same sort order, then we discard it. I'm not sure exactly what the
thresholds should be here, and they probably need to be configurable,
because on a single-user system with excess capacity available it may
be absolutely desirable to use ten times the resources to get an
answer 25% faster, but on a heavy-loaded system that will stink.
Some ideas for GUCs:
max_parallel_degree = The largest number of processes we'll consider
using for a single query.
min_parallel_speedup = The minimum percentage by which a parallel path
must be cheaper (in terms of execution time) than a non-parallel path
in order to survive. I'm imagining the default here might be
something like 15%.
min_parallel_speedup_per_worker = Like the previous one, but per
worker. e.g. if this is 5%, which might be a sensible default, then a
plan with 4 workers must be at least 20% better to survive, but a plan
using only 2 workers only needs to be 10% better.
An additional benefit of this line of thinking is that planning would
always produce a best non-parallel path. And sometimes, there would
also be a best parallel path that is expected to run faster. We could
then choose between them dynamically at execution time.
I think it's pretty hard to imagine a scenario as extreme as the one
you mention above ever actually occurring in practice. I mean, even
the most naive implementation of parallel query will presumably have
something like max_parallel_degree, and you probably won't have that
set to 128. For starters, it can't possibly make sense unless you
server has at least 128 CPUs, and even then it only makes sense if you
don't mind a single query using all of them, and even if the first of
those things is true, the second one probably isn't. I don't doubt
that less extreme variants of this scenario are possible, though.
On 08-04-2015 PM 12:46, Amit Kapila wrote: > Going forward, I think we can improve the same if we decide not to shutdown > parallel workers till postmaster shutdown once they are started and > then just allocate them during executor-start phase. > I wonder if it makes sense to invent the notion of a global pool of workers with configurable number of workers that are created at postmaster start and destroyed at shutdown and requested for use when a query uses parallelizable nodes. That way, parallel costing model might be better able to factor in the available-resources-for-parallelization aspect, too. Though, I'm not quite sure how that helps solve (if at all) the problem of occasional unjustifiable resource consumption due to parallelization. Thanks, Amit
On Wed, Apr 8, 2015 at 1:53 AM, Kevin Grittner <kgrittn@ymail.com> wrote:
>
> David Rowley <dgrowleyml@gmail.com> wrote:
>
> > If we attempt to do this parallel stuff at plan time, and we
> > happen to plan at some quiet period, or perhaps worse, some
> > application's start-up process happens to PREPARE a load of
> > queries when the database is nice and quite, then quite possibly
> > we'll end up with some highly parallel queries. Then perhaps come
> > the time these queries are actually executed the server is very
> > busy... Things will fall apart quite quickly due to the masses of
> > IPC and context switches that would be going on.
> >
> > I completely understand that this parallel query stuff is all
> > quite new to us all and we're likely still trying to nail down
> > the correct infrastructure for it to work well, so this is why
> > I'm proposing that the planner should know nothing of parallel
> > query, instead I think it should work more along the lines of:
> >
> > * Planner should be completely oblivious to what parallel query
> > is.
> > * Before executor startup the plan is passed to a function which
> > decides if we should parallelise it, and does so if the plan
> > meets the correct requirements. This should likely have a very
> > fast exit path such as:
> > if root node's cost < parallel_query_cost_threshold
> > return; /* the query is not expensive enough to attempt to make parallel */
> >
> > The above check will allow us to have an almost zero overhead for
> > small low cost queries.
> >
> > This function would likely also have some sort of logic in order
> > to determine if the server has enough spare resource at the
> > current point in time to allow queries to be parallelised
>
> There is a lot to like about this suggestion.
>
> I've seen enough performance crashes due to too many concurrent
> processes (even when each connection can only use a single process)
> to believe that, for a plan which will be saved, it is possible to
> know at planning time whether parallelization will be a nice win or
> a devastating over-saturation of resources during some later
> execution phase.
>
> Another thing to consider is that this is not entirely unrelated to
> the concept of admission control policies. Perhaps this phase
> could be a more general execution start-up admission control phase,
> where parallel processing would be one adjustment that could be
> considered.I think there is always a chance that resources (like parallel-workers)won't be available at run-time even if we decide about them atexecutor-start phase unless we block it for that node's usage and OTOHif we block it (by allocating) those resources during executor-start phasethen we might end up blocking it too early or may be they won't even getused if we decide not to execute that node. On that basis, it seems tome current strategy is not bad where we decide during planning time andlater during execution time if not all resources (particularly parallel-workers)are not available, then we use only the available one's to execute the plan.Going forward, I think we can improve the same if we decide not to shutdownparallel workers till postmaster shutdown once they are started andthen just allocate them during executor-start phase.
>
> On 8 April 2015 at 15:46, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> I think there is always a chance that resources (like parallel-workers)
>> won't be available at run-time even if we decide about them at
>> executor-start phase unless we block it for that node's usage and OTOH
>> if we block it (by allocating) those resources during executor-start phase
>> then we might end up blocking it too early or may be they won't even get
>> used if we decide not to execute that node. On that basis, it seems to
>> me current strategy is not bad where we decide during planning time and
>> later during execution time if not all resources (particularly parallel-workers)
>> are not available, then we use only the available one's to execute the plan.
>> Going forward, I think we can improve the same if we decide not to shutdown
>> parallel workers till postmaster shutdown once they are started and
>> then just allocate them during executor-start phase.
>>
>>
>
> Yeah, but what about when workers are not available in cases when the plan was only a win because the planner thought there would be lots of workers... There could have been a more optimal serial plan already thrown out by the planner which is no longer available to the executor.
>
On Wed, Apr 8, 2015 at 3:30 PM, David Rowley <dgrowleyml@gmail.com> wrote:
>
> On 8 April 2015 at 15:46, Amit Kapila <amit.kapila16@gmail.com> wrote:
>>
>> I think there is always a chance that resources (like parallel-workers)
>> won't be available at run-time even if we decide about them at
>> executor-start phase unless we block it for that node's usage and OTOH
>> if we block it (by allocating) those resources during executor-start phase
>> then we might end up blocking it too early or may be they won't even get
>> used if we decide not to execute that node. On that basis, it seems to
>> me current strategy is not bad where we decide during planning time and
>> later during execution time if not all resources (particularly parallel-workers)
>> are not available, then we use only the available one's to execute the plan.
>> Going forward, I think we can improve the same if we decide not to shutdown
>> parallel workers till postmaster shutdown once they are started and
>> then just allocate them during executor-start phase.
>>
>>
>
> Yeah, but what about when workers are not available in cases when the plan was only a win because the planner thought there would be lots of workers... There could have been a more optimal serial plan already thrown out by the planner which is no longer available to the executor.
>That could also happen even if we decide in executor-start phase.
I agree that there is a chance of loss incase appropriate resourcesare not available during execution, but same is true for work_memas well for a non-parallel plan. I think we need some advanced wayto handle the case when resources are not available during executionby either re-planing the statement or by some other way, but that canalso be done separately.
On Tue, Apr 7, 2015 at 11:58 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > One disadvantage of retaining parallel-paths could be that it can > increase the number of combinations planner might need to evaluate > during planning (in particular during join path evaluation) unless we > do some special handling to avoid evaluation of such combinations. Yes, that's true. But the overhead might not be very much. In the common case, many baserels and joinrels will have no parallel paths because the non-parallel paths is known to be better anyway. Also, if parallelism does seem to be winning, we're probably planning a query that involves accessing a fair amount of data, so a little extra planner overhead may not be so bad. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Apr 8, 2015 at 3:34 AM, David Rowley <dgrowleyml@gmail.com> wrote: > On 8 April 2015 at 14:24, Robert Haas <robertmhaas@gmail.com> wrote: >> I think one of the philosophical questions that has to be answered >> here is "what does it mean to talk about the cost of a parallel >> plan?". For a non-parallel plan, the cost of the plan means both "the >> amount of effort we will spend executing the plan" and also "the >> amount of time we think the plan will take to complete", but those two >> things are different for parallel plans. I'm inclined to think it's >> right to view the cost of a parallel plan as a proxy for execution >> time, because the fundamental principle of the planner is that we pick >> the lowest-cost plan. But there also clearly needs to be some way to >> prevent the selection of a plan which runs slightly faster at the cost >> of using vastly more resources. > > I'd agree with that as far as CPU costs, or maybe I'd just disagree with the > alternative, as if we costed in <cost of individual worker's work> * <number > of workers> then we'd never choose a parallel plan, as by the time we costed > in tuple communication costs between the processes a parallel plan would > always cost more than the serial equivalent. I/O costs are different, I'd > imagine these shouldn't be divided by the estimated number of workers. It's hard to say. If the I/O is from the OS buffer cache, then there's no reason why several workers can't run in parallel. And even if it's from the actual storage, we don't know what degree of I/O parallelism will be possible. Maybe effective_io_concurrency should play into the costing formula somehow, but it's not very clear to me that captures the information we care about. In general, I'm not sure how common it is for the execution speed of a sequential scan to be limited by I/O. For example, on a pgbench database, scale factor 300, on a POWERPC machine provided by IBM for performance testing (thanks, IBM!) a cached read of the pgbench_accounts files took 1.122 seconds. After dropping the caches, it took 10.427 seconds. "select * from pgbench_accounts where abalance > 30000" took 10.244 seconds with a cold cache and 5.029 seconds with a warm cache. So on this particular hardware, on this particular test, parallelism is useless if the cache is cold, but it could be right to use ~4-5 processes for the scan if the cache is warm. However, we have no way of knowing whether the cache will be cold or warm at execution time. This isn't a new problem. As it is, the user has to set seq_page_cost and random_page_cost based on either a cold-cache assumption or a warm-cache assumption, and if they guess wrong, their costing estimates will be off (on this platform, on this test case) by 4-5x. That's pretty bad, and it's totally unclear to me what to do about it. I'm guessing it's unclear to other people, too, or we would likely have done something about it by now. >> Some ideas for GUCs: >> >> max_parallel_degree = The largest number of processes we'll consider >> using for a single query. >> min_parallel_speedup = The minimum percentage by which a parallel path >> must be cheaper (in terms of execution time) than a non-parallel path >> in order to survive. I'm imagining the default here might be >> something like 15%. >> min_parallel_speedup_per_worker = Like the previous one, but per >> worker. e.g. if this is 5%, which might be a sensible default, then a >> plan with 4 workers must be at least 20% better to survive, but a plan >> using only 2 workers only needs to be 10% better. > > max_parallel_degree feels awfully like it would have to be set > conservatively, similar to how work_mem is today. Like with work_mem, during > quiet periods it sure would be nice if it could magically increase. Absolutely. But, similar to work_mem, that's a really hard problem. We can't know at plan time how much work memory, or how many CPUs, will be available at execution time. And even if we did, it need not be constant throughout the whole of query execution. It could be that when execution starts, there's lots of memory available, so we do a quicksort rather than a tape-sort. But midway through the machine comes under intense memory pressure and there's no way for the system to switch strategies. Now, having said that, I absolutely believe that it's correct for the planner to make the initial decisions in this area. Parallelism changes the cost of execution nodes, and it's completely wrong to assume that this couldn't alter planner decisions at higher levels of the plan tree. At the same time, it's pretty clear that it would be a great thing for the executor to be able to adjust the strategy if the planner's assumptions don't pan out, or if conditions have changed. For example, if we choose a seq-scan-sort-and-filter over an index-scan-and-filter thinking that we'll be able to do a quicksort, and then it turns out that we're short on memory, it's too late to switch gears and adopt the index-scan-and-filter plan after all. That's long since been discarded. But it's still better to switch to a heap sort than to persist with a quicksort that's either going to fail outright, or (maybe worse) succeed but drive the machine into swap, which will just utterly obliterate performance. >> An additional benefit of this line of thinking is that planning would >> always produce a best non-parallel path. And sometimes, there would >> also be a best parallel path that is expected to run faster. We could >> then choose between them dynamically at execution time. > > Actually store 2 plans within the plan? Like with an AlternativePlanNode? Yeah. I'm not positive that's a good idea, but it seems like might be. >> I think it's pretty hard to imagine a scenario as extreme as the one >> you mention above ever actually occurring in practice. I mean, even >> the most naive implementation of parallel query will presumably have >> something like max_parallel_degree, and you probably won't have that >> set to 128. For starters, it can't possibly make sense unless you >> server has at least 128 CPUs, and even then it only makes sense if you >> don't mind a single query using all of them, and even if the first of >> those things is true, the second one probably isn't. I don't doubt >> that less extreme variants of this scenario are possible, though. > > Yeah maybe, it does seem quite extreme, but maybe less so as the years roll > on a bit... perhaps in 5-10 years it might be quite common to have that many > spare CPU cores to throw at a task. That is certainly possible, but we need to start small. It's completely OK for the first version of this feature to have some rough edges that get improved later. Indeed, it's absolutely vital, or we'll never get this thing off the ground. > I think if we have this percentage GUC you mentioned to prefer parallel > plans if they're within a % threshold of the serial plan, then we could end > up with problems with I/O and buffers getting thrown out of caches due to > the extra I/O involved in parallel plans going with seq scans instead of > serial plans choosing index scans. That's possible, but the non-parallel planner doesn't account for caching effects, either. > In summary it sounds like with my idea we get: > > Pros > * Optimal plan if no workers are available at execution time. > * Parallelism possible if the chosen optimal plan happens to support > parallelism, e.g not index scan. > * No planning overhead The third one isn't really true. You've just moved some of the planning to execution time. > Cons: > * The plan "Parallelizer" must make changes to the plan just before > execution time, which ruins the 1 to 1 ratio of plan/executor nodes by the > time you inject Funnel nodes. > > If we parallelise during planning time: > > Pros > * More chance of getting a parallel friendly plan which could end up being > very fast if we get enough workers at executor time. This, to me, is by far the biggest "con" of trying to do something at execution time. If planning doesn't take into account the gains that are possible from parallelism, then you'll only be able to come up with the best parallel plan when it happens to be a parallelized version of the best serial plan. So long as the only parallel operator is parallel seq scan, that will probably be a common scenario. But once we assemble a decent selection of parallel operators, and a reasonably intelligent parallel query optimizer, I'm not so sure it'll still be true. > Cons: > * May produce non optimal plans if no worker processes are available during > execution time. > * Planning overhead for considering parallel paths. > * The parallel plan may blow out buffer caches due to increased I/O of > parallel plan. > > Of course please say if I've missed any pro or con. I think I generally agree with your list; but we might not agree on the relative importance of the items on it. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Apr 8, 2015 at 3:38 AM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp> wrote: > On 08-04-2015 PM 12:46, Amit Kapila wrote: >> Going forward, I think we can improve the same if we decide not to shutdown >> parallel workers till postmaster shutdown once they are started and >> then just allocate them during executor-start phase. > > I wonder if it makes sense to invent the notion of a global pool of workers > with configurable number of workers that are created at postmaster start and > destroyed at shutdown and requested for use when a query uses parallelizable > nodes. Short answer: Yes, but not for the first version of this feature. Longer answer: We can't actually very reasonably have a "global" pool of workers so long as we retain the restriction that a backend connected to one database cannot subsequently disconnect from it and connect to some other database instead. However, it's certainly a good idea to reuse the same workers for subsequent operations on the same database, especially if they are also by the same user. At the very minimum, it would be good to reuse the same workers for subsequent operations within the same query, instead of destroying the old ones and creating new ones. Nonwithstanding the obvious value of all of these ideas, I don't think we should do any of them for the first version of this feature. This is too big a thing to get perfect on the first try. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 2015-04-21 AM 03:29, Robert Haas wrote: > On Wed, Apr 8, 2015 at 3:38 AM, Amit Langote wrote: >> On 08-04-2015 PM 12:46, Amit Kapila wrote: >>> Going forward, I think we can improve the same if we decide not to shutdown >>> parallel workers till postmaster shutdown once they are started and >>> then just allocate them during executor-start phase. >> >> I wonder if it makes sense to invent the notion of a global pool of workers >> with configurable number of workers that are created at postmaster start and >> destroyed at shutdown and requested for use when a query uses parallelizable >> nodes. > > Short answer: Yes, but not for the first version of this feature. > > Longer answer: We can't actually very reasonably have a "global" pool > of workers so long as we retain the restriction that a backend > connected to one database cannot subsequently disconnect from it and > connect to some other database instead. However, it's certainly a > good idea to reuse the same workers for subsequent operations on the > same database, especially if they are also by the same user. At the > very minimum, it would be good to reuse the same workers for > subsequent operations within the same query, instead of destroying the > old ones and creating new ones. Notwithstanding the obvious value of > all of these ideas, I don't think we should do any of them for the > first version of this feature. This is too big a thing to get perfect > on the first try. > Agreed. Perhaps, Amit has worked (is working) on "reuse the same workers for subsequent operations within the same query" Thanks, Amit
On Wed, Apr 8, 2015 at 3:34 AM, David Rowley <dgrowleyml@gmail.com> wrote:
> In summary it sounds like with my idea we get:
>
> Pros
> * Optimal plan if no workers are available at execution time.
> * Parallelism possible if the chosen optimal plan happens to support
> parallelism, e.g not index scan.
> * No planning overhead
The third one isn't really true. You've just moved some of the
planning to execution time.
> Cons:
> * The plan "Parallelizer" must make changes to the plan just before
> execution time, which ruins the 1 to 1 ratio of plan/executor nodes by the
> time you inject Funnel nodes.
>
> If we parallelise during planning time:
>
> Pros
> * More chance of getting a parallel friendly plan which could end up being
> very fast if we get enough workers at executor time.
This, to me, is by far the biggest "con" of trying to do something at
execution time. If planning doesn't take into account the gains that
are possible from parallelism, then you'll only be able to come up
with the best parallel plan when it happens to be a parallelized
version of the best serial plan. So long as the only parallel
operator is parallel seq scan, that will probably be a common
scenario. But once we assemble a decent selection of parallel
operators, and a reasonably intelligent parallel query optimizer, I'm
not so sure it'll still be true.
> Cons:
> * May produce non optimal plans if no worker processes are available during
> execution time.
> * Planning overhead for considering parallel paths.
> * The parallel plan may blow out buffer caches due to increased I/O of
> parallel plan.
>
> Of course please say if I've missed any pro or con.
I think I generally agree with your list; but we might not agree on
the relative importance of the items on it.
I've also been thinking about how, instead of having to have a special PartialSeqScan node which contains a bunch of code to store tuples in a shared memory queue, could we not have a "TupleBuffer", or "ParallelTupleReader" node, one of which would always be the root node of a plan branch that's handed off to a worker process. This node would just try to keep it's shared tuple store full, and perhaps once it fills it could have a bit of a sleep and be woken up when there's a bit more space on the queue. When no more tuples were available from the node below this, then the worker could exit. (providing there was no rescan required)I think between the Funnel node and a ParallelTupleReader we could actually parallelise plans that don't even have parallel safe nodes.... Let me explain:Let's say we have a 4 way join, and the join order must be {a,b}, {c,d} => {a,b,c,d}, Assuming the cost of joining a to b and c to d are around the same, the Parallelizer may notice this and decide to inject a Funnel and then ParallelTupleReader just below the node for c join d and have c join d in parallel. Meanwhile the main worker process could be executing the root node, as normal. This way the main worker wouldn't have to go to the trouble of joining c to d itself as the worker would have done all that hard work.I know the current patch is still very early in the evolution of PostgreSQL's parallel query, but how would that work with the current method of selecting which parts of the plan to parallelise?
I really think the plan needs to be a complete plan before it can be best analysed on how to divide the workload between workers, and also, it would be quite useful to know how many workers are going to be able to lend a hand in order to know best how to divide the plan up as evenly as possible.
> On 2015-04-21 AM 03:29, Robert Haas wrote:
> > On Wed, Apr 8, 2015 at 3:38 AM, Amit Langote wrote:
> >> On 08-04-2015 PM 12:46, Amit Kapila wrote:
> >>> Going forward, I think we can improve the same if we decide not to shutdown
> >>> parallel workers till postmaster shutdown once they are started and
> >>> then just allocate them during executor-start phase.
> >>
> >> I wonder if it makes sense to invent the notion of a global pool of workers
> >> with configurable number of workers that are created at postmaster start and
> >> destroyed at shutdown and requested for use when a query uses parallelizable
> >> nodes.
> >
> > Short answer: Yes, but not for the first version of this feature.
> >
>
> Agreed.
>
> Perhaps, Amit has worked (is working) on "reuse the same workers for
> subsequent operations within the same query"
>
>
> On Tue, Apr 7, 2015 at 11:58 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > One disadvantage of retaining parallel-paths could be that it can
> > increase the number of combinations planner might need to evaluate
> > during planning (in particular during join path evaluation) unless we
> > do some special handling to avoid evaluation of such combinations.
>
> Yes, that's true. But the overhead might not be very much. In the
> common case, many baserels and joinrels will have no parallel paths
> because the non-parallel paths is known to be better anyway. Also, if
> parallelism does seem to be winning, we're probably planning a query
> that involves accessing a fair amount of data,
On Tue, Apr 21, 2015 at 9:38 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Mon, Apr 20, 2015 at 10:08 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> On Tue, Apr 7, 2015 at 11:58 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > One disadvantage of retaining parallel-paths could be that it can >> > increase the number of combinations planner might need to evaluate >> > during planning (in particular during join path evaluation) unless we >> > do some special handling to avoid evaluation of such combinations. >> >> Yes, that's true. But the overhead might not be very much. In the >> common case, many baserels and joinrels will have no parallel paths >> because the non-parallel paths is known to be better anyway. Also, if >> parallelism does seem to be winning, we're probably planning a query >> that involves accessing a fair amount of data, > > Am I understanding right that by above you mean to say that retain the > parallel and non-parallel path only if parallel-path wins over non-parallel > path? Yes. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, Mar 18, 2015 at 11:43 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> I think I figured out the problem. That fix only helps in the case
> >> where the postmaster noticed the new registration previously but
> >> didn't start the worker, and then later notices the termination.
> >> What's much more likely to happen is that the worker is started and
> >> terminated so quickly that both happen before we create a
> >> RegisteredBgWorker for it. The attached patch fixes that case, too.
> >
> > Patch fixes the problem and now for Rescan, we don't need to Wait
> > for workers to finish.
>
> I realized that there is a problem with this. If an error occurs in
> one of the workers just as we're deciding to kill them all, then the
> error won't be reported. Also, the new code to propagate
> XactLastRecEnd won't work right, either. I think we need to find a
> way to shut down the workers cleanly. The idea generally speaking
> should be:
>
> 1. Tell all of the workers that we want them to shut down gracefully
> without finishing the scan.
>
> 2. Wait for them to exit via WaitForParallelWorkersToFinish().
>
> My first idea about how to implement this is to have the master detach
> all of the tuple queues via a new function TupleQueueFunnelShutdown().
> Then, we should change tqueueReceiveSlot() so that it does not throw
> an error when shm_mq_send() returns SHM_MQ_DETACHED. We could modify
> the receiveSlot method of a DestReceiver to return bool rather than
> void; a "true" value can mean "continue processing" where as a "false"
> value can mean "stop early, just as if we'd reached the end of the
> scan".
>
I have implemented this idea (note that I have to expose a new API
assess-parallel-safety-v4.patch [2] (don't forget to run fixpgproc.pl in the patch)
Attachment
On Wed, Apr 22, 2015 at 8:48 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I have implemented this idea (note that I have to expose a new API > shm_mq_from_handle as TupleQueueFunnel stores shm_mq_handle* and > we sum_mq* to call shm_mq_detach) and apart this I have fixed other > problems reported on this thread: > > 1. Execution of initPlan by master backend and then pass the > required PARAM_EXEC parameter values to workers. > 2. Avoid consuming dsm's by freeing the parallel context after > the last tuple is fetched. > 3. Allow execution of Result node in worker backend as that can > be added as a gating filter on top of PartialSeqScan. > 4. Merged parallel heap scan descriptor patch > > To apply the patch, please follow below sequence: > > HEAD Commit-Id: 4d930eee > parallel-mode-v9.patch [1] > assess-parallel-safety-v4.patch [2] (don't forget to run fixpgproc.pl in > the patch) > parallel_seqscan_v14.patch (Attached with this mail) Thanks, this version looks like an improvement. However, I still see some problems: - I believe the separation of concerns between ExecFunnel() and ExecEndFunnel() is not quite right. If the scan is shut down before it runs to completion (e.g. because of LIMIT), then I think we'll call ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I think you probably need to create a static subroutine that is called both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each case cleaning up whatever resources remain. - InitializeParallelWorkers() still mixes together general parallel executor concerns with concerns specific to parallel sequential scan (e.g. EstimatePartialSeqScanSpace). We have to eliminate everything that assumes that what's under a funnel will be, specifically, a partial sequential scan. To make this work properly, I think we should introduce a new function that recurses over the plan tree and invokes some callback for each plan node. I think this could be modeled on this code from ExplainNode(), beginning around line 1593: /* initPlan-s */ if (planstate->initPlan) ExplainSubPlans(planstate->initPlan, ancestors, "InitPlan", es); /* lefttree */ if (outerPlanState(planstate)) ExplainNode(outerPlanState(planstate), ancestors, "Outer", NULL, es); /* righttree */ if (innerPlanState(planstate)) ExplainNode(innerPlanState(planstate), ancestors, "Inner", NULL, es); /* special child plans */ switch (nodeTag(plan)) { /* a bunch of special cases */ } /* subPlan-s */ if (planstate->subPlan) ExplainSubPlans(planstate->subPlan, ancestors, "SubPlan", es); The new function would do the same sort of thing, but instead of explaining each node, it would invoke a callback for each node. Possibly explain.c could use it instead of having hard-coded logic. Possibly it should use the same sort of return-true convention as expression_tree_walker, query_tree_walker, and friends. So let's call it planstate_tree_walker. Now, instead of directly invoking logic specific to parallel sequential scan, it should call planstate_tree_walker() on its lefttree and pass a new function ExecParallelEstimate() as the callback. That function ignores any node that's not parallel aware, but when it sees a partial sequential scan (or, in the future, some a parallel bitmap scan, parallel sort, or what have you) it does the appropriate estimation work. When ExecParallelEstimate() finishes, we InitializeParallelDSM(). Then, we call planstate_tree_walker() on the lefttree again, and this time we pass another new function ExecParallelInitializeDSM(). Like the previous one, that ignores the callbacks from non-parallel nodes, but if it hits a parallel node, then it fills in the parallel bits (i.e. ParallelHeapScanDesc for a partial sequential scan). - shm_mq_from_handle() is probably reasonable, but can we rename it shm_mq_get_queue()? - It's hard to believe this is right: + if (parallelstmt->inst_options) + receiver = None_Receiver; Really? Flush the tuples if there are *any instrumentation options whatsoever*? At the very least, that doesn't look too future-proof, but I'm suspicious that it's outright incorrect. - I think ParallelStmt probably shouldn't be defined in parsenodes.h. That file is included in a lot of places, and adding all of those extra #includes there doesn't seem like a good idea for modularity reasons even if you don't care about partial rebuilds. Something that includes a shm_mq obviously isn't a "parse" node in any meaningful sense anyway. - I don't think you need both setup cost and startup cost. Starting up more workers isn't particularly more expensive than starting up fewer of them, because most of the overhead is in waiting for them to actually start, and the number of workers is reasonable, then they're all be doing that in parallel with each other. I suggest removing parallel_startup_cost and keeping parallel_setup_cost. - In cost_funnel(), I don't think it's right to divide the run cost by nWorkers + 1. Suppose we've got a plan that looks like this: Funnel -> Hash Join -> Partial Seq Scan on a -> Hash -> Seq Scan on b The sequential scan on b is going to get executed once per worker, whereas the effort for the sequential scan on a is going to be divided over all the workers. So the right way to cost this is as follows: (a) The cost of the partial sequential scan on a is equal to the cost of a regular sequential scan, plus a little bit of overhead to account for communication via the ParallelHeapScanDesc, divided by the number of workers + 1. (b) The cost of the remaining nodes under the funnel works normally. (c) The cost of the funnel is equal to the cost of the hash join plus number of tuples multiplied by per-tuple communication overhead plus a large fixed overhead reflecting the time it takes the workers to start. - While create_parallelscan_paths() is quite right to limit the number of workers to no more than the number of pages, it's pretty obvious that in practice that's way too conservative. I suggest we get significantly more aggressive about that, like limiting ourselves to one worker per thousand pages. We don't really know exactly what the costing factors should be here just yet, but we certainly know that spinning up lots of workers to read a handful of pages each must be dumb. And we can save a significant amount of planning time here by not bothering to generate parallel paths for little tiny relations. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, Apr 22, 2015 at 8:48 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I have implemented this idea (note that I have to expose a new API
> > shm_mq_from_handle as TupleQueueFunnel stores shm_mq_handle* and
> > we sum_mq* to call shm_mq_detach) and apart this I have fixed other
> > problems reported on this thread:
> >
> > 1. Execution of initPlan by master backend and then pass the
> > required PARAM_EXEC parameter values to workers.
> > 2. Avoid consuming dsm's by freeing the parallel context after
> > the last tuple is fetched.
> > 3. Allow execution of Result node in worker backend as that can
> > be added as a gating filter on top of PartialSeqScan.
> > 4. Merged parallel heap scan descriptor patch
> >
> > To apply the patch, please follow below sequence:
> >
> > HEAD Commit-Id: 4d930eee
> > parallel-mode-v9.patch [1]
> > assess-parallel-safety-v4.patch [2] (don't forget to run fixpgproc.pl in
> > the patch)
> > parallel_seqscan_v14.patch (Attached with this mail)
>
> Thanks, this version looks like an improvement. However, I still see
> some problems:
>
> - I believe the separation of concerns between ExecFunnel() and
> ExecEndFunnel() is not quite right. If the scan is shut down before
> it runs to completion (e.g. because of LIMIT), then I think we'll call
> ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I
> think you probably need to create a static subroutine that is called
> both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each
> case cleaning up whatever resources remain.
>
> - InitializeParallelWorkers() still mixes together general parallel
> executor concerns with concerns specific to parallel sequential scan
> (e.g. EstimatePartialSeqScanSpace).
> - shm_mq_from_handle() is probably reasonable, but can we rename it
> shm_mq_get_queue()?
>
> - It's hard to believe this is right:
>
> + if (parallelstmt->inst_options)
> + receiver = None_Receiver;
>
> Really? Flush the tuples if there are *any instrumentation options
> whatsoever*? At the very least, that doesn't look too future-proof,
> but I'm suspicious that it's outright incorrect.
>
> - I think ParallelStmt probably shouldn't be defined in parsenodes.h.
> That file is included in a lot of places, and adding all of those
> extra #includes there doesn't seem like a good idea for modularity
> reasons even if you don't care about partial rebuilds. Something that
> includes a shm_mq obviously isn't a "parse" node in any meaningful
> sense anyway.
>
> - I don't think you need both setup cost and startup cost. Starting
> up more workers isn't particularly more expensive than starting up
> fewer of them, because most of the overhead is in waiting for them to
> actually start, and the number of workers is reasonable, then they're
> all be doing that in parallel with each other. I suggest removing
> parallel_startup_cost and keeping parallel_setup_cost.
>
> - In cost_funnel(), I don't think it's right to divide the run cost by
> nWorkers + 1. Suppose we've got a plan that looks like this:
>
> Funnel
> -> Hash Join
> -> Partial Seq Scan on a
> -> Hash
> -> Seq Scan on b
>
> The sequential scan on b is going to get executed once per worker,
> whereas the effort for the sequential scan on a is going to be divided
> over all the workers. So the right way to cost this is as follows:
>
> (a) The cost of the partial sequential scan on a is equal to the cost
> of a regular sequential scan, plus a little bit of overhead to account
> for communication via the ParallelHeapScanDesc, divided by the number
> of workers + 1.
> (b) The cost of the remaining nodes under the funnel works normally.
> (c) The cost of the funnel is equal to the cost of the hash join plus
> number of tuples multiplied by per-tuple communication overhead plus a
> large fixed overhead reflecting the time it takes the workers to
> start.
>
> - While create_parallelscan_paths() is quite right to limit the number
> of workers to no more than the number of pages, it's pretty obvious
> that in practice that's way too conservative. I suggest we get
> significantly more aggressive about that, like limiting ourselves to
> one worker per thousand pages. We don't really know exactly what the
> costing factors should be here just yet, but we certainly know that
> spinning up lots of workers to read a handful of pages each must be
> dumb. And we can save a significant amount of planning time here by
> not bothering to generate parallel paths for little tiny relations.
>
makes sense, will change.
On Fri, Apr 24, 2015 at 8:32 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> - InitializeParallelWorkers() still mixes together general parallel >> executor concerns with concerns specific to parallel sequential scan >> (e.g. EstimatePartialSeqScanSpace). > > Here we are doing 2 things, first one is for planned statement and > then second one is node specific which in the case is parallelheapscan > descriptor. So If I understand correctly, you want that we remove second > one and have a recursive function to achieve the same. Right. >> - It's hard to believe this is right: >> >> + if (parallelstmt->inst_options) >> + receiver = None_Receiver; >> >> Really? Flush the tuples if there are *any instrumentation options >> whatsoever*? At the very least, that doesn't look too future-proof, >> but I'm suspicious that it's outright incorrect. > > instrumentation info is for explain statement where we don't need > tuples and it is set same way for it as well, refer ExplainOnePlan(). > What makes you feel this is incorrect? Well, for one thing, it's going to completely invalidate the result of EXPLAIN. I mean, consider this: Hash Join -> Parallel Seq Scan -> Hash -> Seq Scan If you have the workers throw away the rows from the parallel seq scan instead of sending them back to the master, the master won't join those rows against the other table. And then the "actual" row counts, timing, etc. will all be totally wrong. Worse, if the user is EXPLAIN-ing a SELECT INTO command, the results will be totally wrong. I don't think you can use ExplainOnePlan() as precedent for the theory that explain_options != 0 means discard everything, because that function does not do that. It bases the decision to throw away the output on the fact that EXPLAIN was used, and throws it away unless an IntoClause was also specified. It does this even if instrument_options == 0. Meanwhile, auto_explain does NOT throw away the output even if instrument_options != 0, nor should it! But even if none of that were an issue, throwing away part of the results from an internal plan tree is not the same thing as throwing away the final result stream, and is dead wrong. >> - I think ParallelStmt probably shouldn't be defined in parsenodes.h. >> That file is included in a lot of places, and adding all of those >> extra #includes there doesn't seem like a good idea for modularity >> reasons even if you don't care about partial rebuilds. Something that >> includes a shm_mq obviously isn't a "parse" node in any meaningful >> sense anyway. > > How about tcop/tcopprot.h? The comment of that file is "prototypes for postgres.c". Generally, unless there is some reason to do otherwise, the prototypes for a .c file in src/backend go in a .h file with the same name in src/include. I don't see why we should do differently here. ParallelStmt should be defined and used in a file living in src/backend/executor, and the header should have the same name and go in src/include/executor. >> - I don't think you need both setup cost and startup cost. Starting >> up more workers isn't particularly more expensive than starting up >> fewer of them, because most of the overhead is in waiting for them to >> actually start, and the number of workers is reasonable, then they're >> all be doing that in parallel with each other. I suggest removing >> parallel_startup_cost and keeping parallel_setup_cost. > > There is some work (like creation of shm queues, launching of workers) > which is done proportional to number of workers during setup time. I > have kept 2 parameters to distinguish such work. I think you have a > point that start of some or all workers could be parallel, but I feel > that still is a work proportinal to number of workers. For future > parallel operations also such a parameter could be useful where we need > to setup IPC between workers or some other stuff where work is proportional > to workers. That's technically true, but the incremental work involved in supporting a new worker is extremely small compare with worker startup times. I'm guessing that the setup cost is going to be on the order of hundred-thousands or millions and and the startup cost is going to be on the order of tens or ones. Unless you can present some contrary evidence, I think we should rip it out. And I actually hope you *can't* present some contrary evidence. Because if you can, then that might mean that we need to cost every possible path from 0 up to N workers and let the costing machinery decide which one is better. If you can't, then we can cost the non-parallel path and the maximally-parallel path and be done. And that would be much better, because it will be faster. Remember, just because we cost a bunch of parallel paths doesn't mean that any of them will actually be chosen. We need to avoid generating too much additional planner work in cases where we don't end up deciding on parallelism anyway. >> - In cost_funnel(), I don't think it's right to divide the run cost by >> nWorkers + 1. Suppose we've got a plan that looks like this: >> >> Funnel >> -> Hash Join >> -> Partial Seq Scan on a >> -> Hash >> -> Seq Scan on b >> >> The sequential scan on b is going to get executed once per worker, >> whereas the effort for the sequential scan on a is going to be divided >> over all the workers. So the right way to cost this is as follows: >> >> (a) The cost of the partial sequential scan on a is equal to the cost >> of a regular sequential scan, plus a little bit of overhead to account >> for communication via the ParallelHeapScanDesc, divided by the number >> of workers + 1. >> (b) The cost of the remaining nodes under the funnel works normally. >> (c) The cost of the funnel is equal to the cost of the hash join plus >> number of tuples multiplied by per-tuple communication overhead plus a >> large fixed overhead reflecting the time it takes the workers to >> start. >> > > IIUC, the change for this would be to remove the change related to > run cost (divide the run cost by nWorkers + 1) from cost_funnel > and made similar change as suggested by point (a) in cost calculation > of partial sequence scan. Right. > As of now, we don't do anything which can > move Funnel node on top of hash join, so not sure if you are expecting > any extra handling as part of point (b) or (c). But we will want to do that in the future, so we should set up the costing correctly now. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Apr 24, 2015 at 8:32 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>> ExecEndFunnel() is not quite right. If the scan is shut down before
>> it runs to completion (e.g. because of LIMIT), then I think we'll call
>> ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I
>> think you probably need to create a static subroutine that is called
>> both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each
>> case cleaning up whatever resources remain.
>>
> >> - I don't think you need both setup cost and startup cost. Starting
> >> up more workers isn't particularly more expensive than starting up
> >> fewer of them, because most of the overhead is in waiting for them to
> >> actually start, and the number of workers is reasonable, then they're
> >> all be doing that in parallel with each other. I suggest removing
> >> parallel_startup_cost and keeping parallel_setup_cost.
> >
> > There is some work (like creation of shm queues, launching of workers)
> > which is done proportional to number of workers during setup time. I
> > have kept 2 parameters to distinguish such work. I think you have a
> > point that start of some or all workers could be parallel, but I feel
> > that still is a work proportinal to number of workers. For future
> > parallel operations also such a parameter could be useful where we need
> > to setup IPC between workers or some other stuff where work is proportional
> > to workers.
>
> That's technically true, but the incremental work involved in
> supporting a new worker is extremely small compare with worker startup
> times. I'm guessing that the setup cost is going to be on the order
> of hundred-thousands or millions and and the startup cost is going to
> be on the order of tens or ones.
> And I actually hope you *can't* present some contrary evidence.
> Because if you can, then that might mean that we need to cost every
> possible path from 0 up to N workers and let the costing machinery
> decide which one is better.
On Wed, May 6, 2015 at 7:55 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >>> - I believe the separation of concerns between ExecFunnel() and >>> ExecEndFunnel() is not quite right. If the scan is shut down before >>> it runs to completion (e.g. because of LIMIT), then I think we'll call >>> ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I >>> think you probably need to create a static subroutine that is called >>> both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each >>> case cleaning up whatever resources remain. > >> Right, will fix as per suggestion. > > I observed one issue while working on this review comment. When we > try to destroy the parallel setup via ExecEndNode (as due to Limit > Node, it could not destroy after consuming all tuples), it waits for > parallel > workers to finish (WaitForParallelWorkersToFinish()) and parallel workers > are waiting for master backend to signal them as their queue is full. > I think in such a case master backend needs to inform workers either when > the scan is discontinued due to limit node or while waiting for parallel > workers to finish. Isn't this why TupleQueueFunnelShutdown() calls shm_mq_detach()? That's supposed to unstick the workers; any impending or future writes will just return SHM_MQ_DETACHED without waiting. >> That's technically true, but the incremental work involved in >> supporting a new worker is extremely small compare with worker startup >> times. I'm guessing that the setup cost is going to be on the order >> of hundred-thousands or millions and and the startup cost is going to >> be on the order of tens or ones. > > Can we safely estimate the cost of restoring parallel state (GUC's, > combo CID, transaction state, snapshot, etc.) in each worker as a setup > cost? There could be some work like restoration of locks (acquire all or > relevant locks at start of parallel worker, if we follow your proposed > design > and even if we don't follow that there could be some similar substantial > work) > which could be substantial and we need to do the same for each worker. > If you think restoration of parallel state in each worker is a pretty > small work, then what you say makes sense to me. Well, all the workers restore that state in parallel, so adding it up across all workers doesn't really make sense. But anyway, no, I don't think that's a big cost. I think the big cost is going to the operating system overhead of process creation. The new process will incur lots of page faults as it populates its address space and dirties pages marked copy-on-write. That's where I expect most of the expense to be. >> And I actually hope you *can't* present some contrary evidence. >> Because if you can, then that might mean that we need to cost every >> possible path from 0 up to N workers and let the costing machinery >> decide which one is better. > > Not necesarally, we can follow a rule that number of workers > that need to be used for any parallel statement are equal to degree of > parallelism (parallel_seqscan_degree) as set by user. I think we > need to do some split up of number workers when there are multiple > parallel operations in single statement (like sort and parallel scan). Yeah. I'm hoping we will be able to use the same pool of workers for multiple operations, but I realize that's a feature we haven't designed yet. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, May 6, 2015 at 7:55 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >>> - I believe the separation of concerns between ExecFunnel() and
> >>> ExecEndFunnel() is not quite right. If the scan is shut down before
> >>> it runs to completion (e.g. because of LIMIT), then I think we'll call
> >>> ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I
> >>> think you probably need to create a static subroutine that is called
> >>> both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each
> >>> case cleaning up whatever resources remain.
> >
> >> Right, will fix as per suggestion.
> >
> > I observed one issue while working on this review comment. When we
> > try to destroy the parallel setup via ExecEndNode (as due to Limit
> > Node, it could not destroy after consuming all tuples), it waits for
> > parallel
> > workers to finish (WaitForParallelWorkersToFinish()) and parallel workers
> > are waiting for master backend to signal them as their queue is full.
> > I think in such a case master backend needs to inform workers either when
> > the scan is discontinued due to limit node or while waiting for parallel
> > workers to finish.
>
> Isn't this why TupleQueueFunnelShutdown() calls shm_mq_detach()?
> That's supposed to unstick the workers; any impending or future writes
> will just return SHM_MQ_DETACHED without waiting.
>
Okay, that can work if we call it in ExecEndNode() before
>
> Well, all the workers restore that state in parallel, so adding it up
> across all workers doesn't really make sense. But anyway, no, I don't
> think that's a big cost. I think the big cost is going to the
> operating system overhead of process creation. The new process will
> incur lots of page faults as it populates its address space and
> dirties pages marked copy-on-write. That's where I expect most of the
> expense to be.
>
Okay, will remove parallel_startup_cost from patch in next version.
On Thu, May 7, 2015 at 3:23 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> > I observed one issue while working on this review comment. When we >> > try to destroy the parallel setup via ExecEndNode (as due to Limit >> > Node, it could not destroy after consuming all tuples), it waits for >> > parallel >> > workers to finish (WaitForParallelWorkersToFinish()) and parallel >> > workers >> > are waiting for master backend to signal them as their queue is full. >> > I think in such a case master backend needs to inform workers either >> > when >> > the scan is discontinued due to limit node or while waiting for parallel >> > workers to finish. >> >> Isn't this why TupleQueueFunnelShutdown() calls shm_mq_detach()? >> That's supposed to unstick the workers; any impending or future writes >> will just return SHM_MQ_DETACHED without waiting. > > Okay, that can work if we call it in ExecEndNode() before > WaitForParallelWorkersToFinish(), however what if we want to do something > like TupleQueueFunnelShutdown() when Limit node decides to stop processing > the outer node. We can traverse the whole plan tree and find the nodes > where > parallel workers needs to be stopped, but I don't think thats good way to > handle > it. If we don't want to stop workers from processing until > ExecutorEnd()--->ExecEndNode(), then it will lead to workers continuing till > that time and it won't be easy to get instrumentation/buffer usage > information > from workers (workers fill such information for master backend after > execution > is complete) as that is done before ExecutorEnd(). For Explain Analyze .., > we > can ensure that workers are stopped before fetching that information from > Funnel node, but the same is not easy for buffer usage stats required by > plugins as that operates at ExecutorRun() and ExecutorFinish() level where > we don't have direct access to node level information. You can refer > pgss_ExecutorEnd() where it completes the storage of stats information > before calling ExecutorEnd(). Offhand, I could not think of a good way to > do this, but one crude way could be introduce a new API > (ParallelExecutorEnd()) > for such plugins which needs to be called before completing the stats > accumulation. > This API will call ExecEndPlan() if parallelmodeNeeded flag is set and allow > accumulation of stats (InstrStartNode()/InstrStopNode()) OK, so if I understand you here, the problem is what to do about an "orphaned" worker. The Limit node just stops fetching from the lower nodes, and those nodes don't get any clue that this has happened, so their workers just sit there until the end of the query. Of course, that happens already, but it doesn't usually hurt very much, because the Limit node usually appears at or near the top of the plan. It could matter, though. Suppose the Limit is for a subquery that has a Sort somewhere (not immediately) beneath it. My guess is the Sort's tuplestore will stick around until after the subquery finishes executing for as long as the top-level query is executing, which in theory could be a huge waste of resources. In practice, I guess people don't really write queries that way. If they did, I think we'd have already developed some general method for fixing this sort of problem. I think it might be better to try to solve this problem in a more localized way. Can we arrange for planstate->instrumentation to point directory into the DSM, instead of copying the data over later? That seems like it might help, or perhaps there's another approach. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Apr 22, 2015 at 10:48 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > parallel_seqscan_v14.patch (Attached with this mail) This patch is not applying/working with the latest head after parallel mode patch got committed. can you please rebase the patch. Regards, Hari Babu Fujitsu Australia
>
> On Wed, Apr 22, 2015 at 10:48 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > parallel_seqscan_v14.patch (Attached with this mail)
>
> This patch is not applying/working with the latest head after parallel
> mode patch got committed.
> can you please rebase the patch.
>
Thanks for reminding, I am planing to work on remaining review
>
>
> I think it might be better to try to solve this problem in a more
> localized way. Can we arrange for planstate->instrumentation to point
> directory into the DSM, instead of copying the data over later?
On Tue, May 19, 2015 at 8:45 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Mon, May 11, 2015 at 3:00 AM, Robert Haas <robertmhaas@gmail.com> wrote: >> I think it might be better to try to solve this problem in a more >> localized way. Can we arrange for planstate->instrumentation to point >> directory into the DSM, instead of copying the data over later? > > Yes, we can do that but I am not sure we can do that for pgBufferUsage > which is a separate information we need to pass back to master backend. > One way could be to change pgBufferUsage to a pointer and then allocate > the memory for same at backend startup time and for parallel workers, it > should point to DSM. Do you see any simple way to handle it? No, that seems problematic. > Another way could be that master backend waits for parallel workers to > finish before collecting the instrumentation information and buffer usage > stats. It seems to me that we need this information (stats) after execution > in master backend is over, so I think we can safely assume that it is okay > to finish the execution of parallel workers if they are not already finished > the execution. I'm not sure exactly where you plan to insert the wait. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, Apr 22, 2015 at 8:48 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I have implemented this idea (note that I have to expose a new API
> > shm_mq_from_handle as TupleQueueFunnel stores shm_mq_handle* and
> > we sum_mq* to call shm_mq_detach) and apart this I have fixed other
> > problems reported on this thread:
> >
> > 1. Execution of initPlan by master backend and then pass the
> > required PARAM_EXEC parameter values to workers.
> > 2. Avoid consuming dsm's by freeing the parallel context after
> > the last tuple is fetched.
> > 3. Allow execution of Result node in worker backend as that can
> > be added as a gating filter on top of PartialSeqScan.
> > 4. Merged parallel heap scan descriptor patch
> >
> > To apply the patch, please follow below sequence:
> >
> > HEAD Commit-Id: 4d930eee
> > parallel-mode-v9.patch [1]
> > assess-parallel-safety-v4.patch [2] (don't forget to run fixpgproc.pl in
> > the patch)
> > parallel_seqscan_v14.patch (Attached with this mail)
>
> Thanks, this version looks like an improvement. However, I still see
> some problems:
>
> - I believe the separation of concerns between ExecFunnel() and
> ExecEndFunnel() is not quite right. If the scan is shut down before
> it runs to completion (e.g. because of LIMIT), then I think we'll call
> ExecEndFunnel() before ExecFunnel() hits the TupIsNull(slot) path. I
> think you probably need to create a static subroutine that is called
> both as soon as TupIsNull(slot) and also from ExecEndFunnel(), in each
> case cleaning up whatever resources remain.
>
> - InitializeParallelWorkers() still mixes together general parallel
> executor concerns with concerns specific to parallel sequential scan
> (e.g. EstimatePartialSeqScanSpace). We have to eliminate everything
> that assumes that what's under a funnel will be, specifically, a
> partial sequential scan.
>
>
> - shm_mq_from_handle() is probably reasonable, but can we rename it
> shm_mq_get_queue()?
>
> - It's hard to believe this is right:
>
> + if (parallelstmt->inst_options)
> + receiver = None_Receiver;
>
> Really? Flush the tuples if there are *any instrumentation options
> whatsoever*? At the very least, that doesn't look too future-proof,
> but I'm suspicious that it's outright incorrect.
>
> - I think ParallelStmt probably shouldn't be defined in parsenodes.h.
> That file is included in a lot of places, and adding all of those
> extra #includes there doesn't seem like a good idea for modularity
> reasons even if you don't care about partial rebuilds. Something that
> includes a shm_mq obviously isn't a "parse" node in any meaningful
> sense anyway.
>
> - I don't think you need both setup cost and startup cost. Starting
> up more workers isn't particularly more expensive than starting up
> fewer of them, because most of the overhead is in waiting for them to
> actually start, and the number of workers is reasonable, then they're
> all be doing that in parallel with each other. I suggest removing
> parallel_startup_cost and keeping parallel_setup_cost.
>
> - In cost_funnel(), I don't think it's right to divide the run cost by
> nWorkers + 1. Suppose we've got a plan that looks like this:
>
> Funnel
> -> Hash Join
> -> Partial Seq Scan on a
> -> Hash
> -> Seq Scan on b
>
> The sequential scan on b is going to get executed once per worker,
> whereas the effort for the sequential scan on a is going to be divided
> over all the workers. So the right way to cost this is as follows:
>
> (a) The cost of the partial sequential scan on a is equal to the cost
> of a regular sequential scan, plus a little bit of overhead to account
> for communication via the ParallelHeapScanDesc, divided by the number
> of workers + 1.
> (b) The cost of the remaining nodes under the funnel works normally.
> (c) The cost of the funnel is equal to the cost of the hash join plus
> number of tuples multiplied by per-tuple communication overhead plus a
> large fixed overhead reflecting the time it takes the workers to
> start.
>
> - While create_parallelscan_paths() is quite right to limit the number
> of workers to no more than the number of pages, it's pretty obvious
> that in practice that's way too conservative. I suggest we get
> significantly more aggressive about that, like limiting ourselves to
> one worker per thousand pages. We don't really know exactly what the
> costing factors should be here just yet, but we certainly know that
> spinning up lots of workers to read a handful of pages each must be
> dumb. And we can save a significant amount of planning time here by
> not bothering to generate parallel paths for little tiny relations.
>
Attachment
[Jumping in without catching up on entire thread. Please let me know if these questions have already been covered.] 1. Can you change the name to something like ParallelHeapScan? Parallel Sequential is a contradiction. (I know this is bikeshedding and I won't protest further if you keep the name.) 2. Where is the speedup coming from? How much of it is CPU and IO overlapping (i.e. not leaving disk or CPU idle while the other is working), and how much from the CPU parallelism? I know this is difficult to answer rigorously, but it would be nice to have some breakdown even if for a specific machine. Regards, Jeff Davis
>
> [Jumping in without catching up on entire thread.
> if these questions have already been covered.]
>
> 1. Can you change the name to something like ParallelHeapScan?
> Parallel Sequential is a contradiction. (I know this is bikeshedding
> and I won't protest further if you keep the name.)
>
> 2. Where is the speedup coming from? How much of it is CPU and IO
> overlapping (i.e. not leaving disk or CPU idle while the other is
> working), and how much from the CPU parallelism? I know this is
> difficult to answer rigorously, but it would be nice to have some
> breakdown even if for a specific machine.
>
On 2015-07-01 PM 02:37, Amit Kapila wrote: > > In terms of completeness, I think we should add some documentation > for this patch, one way is to update about the execution mechanism in > src/backend/access/transam/README.parallel and then explain about > new configuration knobs in documentation (.sgml files). Also we > can have a separate page in itself in documentation under Server > Programming Section (Parallel Query -> Parallel Scan; > Parallel Scan Examples; ...) > > Another thing to think about this patch at this stage do we need to > breakup this patch and if yes, how to break it up into multiple patches, > so that it can be easier to complete the review. I could see that it > can be splitted into 2 or 3 patches. > a. Infrastructure for parallel execution, like some of the stuff in > execparallel.c, heapam.c,tqueue.c, etc and all other generic > (non-nodes specific) code. > b. Nodes (Funnel and PartialSeqScan) specific code for optimiser > and executor. > c. Documentation > > Suggestions? A src/backend/executor/README.parallel? Thanks, Amit
On Wed, 2015-07-01 at 11:07 +0530, Amit Kapila wrote: > For what you are asking to change name for? There are still some places, at least in the comments, that call it a parallel sequential scan. > a. Infrastructure for parallel execution, like some of the stuff in > execparallel.c, heapam.c,tqueue.c, etc and all other generic > (non-nodes specific) code. Did you consider passing tuples through the tqueue by reference rather than copying? The page should be pinned by the worker process, but perhaps that's a bad assumption to make? Regards,Jeff Davis
> > a. Infrastructure for parallel execution, like some of the stuff in > > execparallel.c, heapam.c,tqueue.c, etc and all other generic > > (non-nodes specific) code. > > Did you consider passing tuples through the tqueue by reference rather > than copying? The page should be pinned by the worker process, but > perhaps that's a bad assumption to make? > Is the upcoming PartialAggregate/FinalAggregate a solution for the problem? More or less, the Funnel node run on single core has to process massive amount of tuples that are fetched in parallel. Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com> > -----Original Message----- > From: Jeff Davis [mailto:pgsql@j-davis.com] > Sent: Wednesday, July 01, 2015 4:51 PM > To: Amit Kapila > Cc: Robert Haas; Haribabu Kommi; Andres Freund; Kaigai Kouhei(海外 浩平); Amit > Langote; Amit Langote; Fabrízio Mello; Thom Brown; Stephen Frost; pgsql-hackers > Subject: Re: [HACKERS] Parallel Seq Scan > > On Wed, 2015-07-01 at 11:07 +0530, Amit Kapila wrote: > > > For what you are asking to change name for? > > There are still some places, at least in the comments, that call it a > parallel sequential scan. > > > > a. Infrastructure for parallel execution, like some of the stuff in > > execparallel.c, heapam.c,tqueue.c, etc and all other generic > > (non-nodes specific) code. > > Did you consider passing tuples through the tqueue by reference rather > than copying? The page should be pinned by the worker process, but > perhaps that's a bad assumption to make? > > Regards, > Jeff Davis >
>
> On Wed, 2015-07-01 at 11:07 +0530, Amit Kapila wrote:
>
> > For what you are asking to change name for?
>
> There are still some places, at least in the comments, that call it a
> parallel sequential scan.
>
>
> > a. Infrastructure for parallel execution, like some of the stuff in
> > execparallel.c, heapam.c,tqueue.c, etc and all other generic
> > (non-nodes specific) code.
>
> Did you consider passing tuples through the tqueue by reference rather
> than copying? The page should be pinned by the worker process, but
> perhaps that's a bad assumption to make?
>
On 01/07/15 17:37, Amit Kapila wrote: > On Tue, Jun 30, 2015 at 4:00 AM, Jeff Davis <pgsql@j-davis.com > <mailto:pgsql@j-davis.com>> wrote: > > > > [Jumping in without catching up on entire thread. > [...] > . > > > 2. Where is the speedup coming from? How much of it is CPU and IO > > overlapping (i.e. not leaving disk or CPU idle while the other is > > working), and how much from the CPU parallelism? I know this is > > difficult to answer rigorously, but it would be nice to have some > > breakdown even if for a specific machine. > > > > Yes, you are right and we have done quite some testing (on the hardware > available) with this patch (with different approaches) to see how much > difference it creates for IO and CPU, with respect to IO we have found > that it doesn't help much [1], though it helps when the data is cached > and there are really good benefits in terms of CPU [2]. > [...] I assume your answer refers to a table on one spindle of spinning rust. QUESTIONS: 1. what about I/O using an SSD? 2. what if the table is in a RAID array (of various types), would having the table spread over multiple spindles help? Cheers, Gavin
>
> On 01/07/15 17:37, Amit Kapila wrote:
>>
>> Yes, you are right and we have done quite some testing (on the hardware
>> available) with this patch (with different approaches) to see how much
>> difference it creates for IO and CPU, with respect to IO we have found
>> that it doesn't help much [1], though it helps when the data is cached
>> and there are really good benefits in terms of CPU [2].
>>
> [...]
>
> I assume your answer refers to a table on one spindle of spinning rust.
>
>
> QUESTIONS:
>
> 1. what about I/O using an SSD?
>
> 2. what if the table is in a RAID array (of various types), would
> having the table spread over multiple spindles help?
>
I think it will be helpful if we could get the numbers on more type of m/c's,
Attachment
On Fri, 2015-07-03 at 17:35 +0530, Amit Kapila wrote: > Attached, find the rebased version of patch. > Comments: * The heapam.c changes seem a little ad-hoc. Conceptually, which portions should be affected by parallelism? How do we know we didn't miss something? * Why is initscan getting the number of blocks from the structure? Is it just to avoid an extra syscall, or is there a correctness issue there? Is initscan expecting that heap_parallelscan_initialize is always called first (if parallel)? Please add a comment explaining above. * What's the difference between scan->rs_nblocks and scan->rs_parallel->phs_nblocks? Same for rs_rd->rd_id and phs_relid. * It might be good to separate out some fields which differ between the normal heap scan and the parallel heap scan. Perhaps put rs_ctup, rs_cblock, and rs_cbuf into a separate structure, which is always NULL during a parallel scan. That way we don't accidentally use a non-parallel field when doing a parallel scan. * Is there a reason that partial scans can't work with syncscan? It looks like you're not choosing the starting block in the same way, so it always starts at zero and never does syncscan. If we don't want to mix syncscan and partial scan, that's fine, but it should be more explicit. I'm trying to understand where tqueue.c fits in. It seems very closely tied to the Funnel operator, because any change to the way Funnel works would almost certainly require changes in tqueue.c. But "tqueue" is a generic name for the file, so something seems off. Either we should explicitly make it the supporting routines for the Funnel operator, or we should try to generalize it a little. I still have quite a bit to look at, but this is a start. Regards,Jeff Davis
>
> On Fri, 2015-07-03 at 17:35 +0530, Amit Kapila wrote:
>
> > Attached, find the rebased version of patch.
> >
>
> Comments:
>
>
> * The heapam.c changes seem a little ad-hoc. Conceptually, which
> portions should be affected by parallelism? How do we know we didn't
> miss something?
> * Why is initscan getting the number of blocks from the structure? Is it
> just to avoid an extra syscall, or is there a correctness issue there?
> Is initscan expecting that heap_parallelscan_initialize is always called
> first (if parallel)? Please add a comment explaining above.
> * What's the difference between scan->rs_nblocks and
> scan->rs_parallel->phs_nblocks?
> * It might be good to separate out some fields which differ between the
> normal heap scan and the parallel heap scan. Perhaps put rs_ctup,
> rs_cblock, and rs_cbuf into a separate structure, which is always NULL
> during a parallel scan. That way we don't accidentally use a
> non-parallel field when doing a parallel scan.
> * Is there a reason that partial scans can't work with syncscan? It
> looks like you're not choosing the starting block in the same way, so it
> always starts at zero and never does syncscan.
> syncscan and partial scan, that's fine, but it should be more explicit.
>
> tied to the Funnel operator, because any change to the way Funnel works
> would almost certainly require changes in tqueue.c.
> generic name for the file, so something seems off. Either we should
> explicitly make it the supporting routines for the Funnel operator, or
> we should try to generalize it a little.
>
> I still have quite a bit to look at, but this is a start.
>
Thanks for the review.
On Mon, 2015-07-06 at 10:37 +0530, Amit Kapila wrote: > Or the other way to look at it could be separate out fields which are > required for parallel scan which is done currently by forming a > separate structure ParallelHeapScanDescData. > I was suggesting that you separate out both the normal scan fields and the partial scan fields, that way we're sure that rs_nblocks is not accessed during a parallel scan. Or, you could try wrapping the parts of heapam.c that are affected by parallelism into new static functions. > The reason why partial scan can't be mixed with sync scan is that in > parallel > scan, it performs the scan of heap by synchronizing blocks (each > parallel worker > scans a block and then asks for a next block to scan) among parallel > workers. > Now if we try to make sync scans work along with it, the > synchronization among > parallel workers will go for a toss. It might not be impossible to > make that > work in some way, but not sure if it is important enough for sync > scans to work > along with parallel scan. I haven't tested it, but I think it would still be helpful. The block accesses are still in order even during a partial scan, so why wouldn't it help? You might be concerned about the reporting of a block location, which would become more noisy with increased parallelism. But in my original testing, sync scans weren't very sensitive to slight deviations, because of caching effects. > tqueue.c is mainly designed to pass tuples between parallel workers > and currently it is used in Funnel operator to gather the tuples > generated > by all the parallel workers. I think we can use it for any other > operator > which needs tuple communication among parallel workers. Some specifics of the Funnel operator seem to be a part of tqueue, which doesn't make sense to me. For instance, reading from the set of queues in a round-robin fashion is part of the Funnel algorithm, and doesn't seem suitable for a generic tuple communication mechanism (that would never allow order-sensitive reading, for example). Regards,Jeff Davis
On Fri, Jul 3, 2015 at 10:05 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached, find the rebased version of patch. > > Note - You need to first apply the assess-parallel-safety patch which you > can find at: > http://www.postgresql.org/message-id/CAA4eK1JjsfE_dOsHTr_z1P_cBKi_X4C4X3d7Nv=VWX9fs7qdJA@mail.gmail.com I ran some performance tests on a 16 core machine with large shared buffers, so there is no IO involved. With the default value of cpu_tuple_comm_cost, parallel plan is not getting generated even if we are selecting 100K records from 40 million records. So I changed the value to '0' and collected the performance readings. Here are the performance numbers: selectivity(millions) Seq scan(ms) Parallel scan 2 workers 4 workers 8 workers 0.1 11498.93 4821.40 3305.84 3291.90 0.4 10942.98 4967.46 3338.58 3374.00 0.8 11619.44 5189.61 3543.86 3534.40 1.5 12585.51 5718.07 4162.71 2994.90 2.7 14725.66 8346.96 10429.05 8049.11 5.4 18719.00 20212.33 21815.19 19026.99 7.2 21955.79 28570.74 28217.60 27042.27 The average table row size is around 500 bytes and query selection column width is around 36 bytes. when the query selectivity goes more than 10% of total table records, the parallel scan performance is dropping. Regards, Hari Babu Fujitsu Australia
>
> On Mon, 2015-07-06 at 10:37 +0530, Amit Kapila wrote:
>
> > Or the other way to look at it could be separate out fields which are
> > required for parallel scan which is done currently by forming a
> > separate structure ParallelHeapScanDescData.
> >
> I was suggesting that you separate out both the normal scan fields and
> the partial scan fields, that way we're sure that rs_nblocks is not
> accessed during a parallel scan.
>
> Or, you could try wrapping the parts of heapam.c that are affected by
> parallelism into new static functions.
>
> > The reason why partial scan can't be mixed with sync scan is that in
> > parallel
> > scan, it performs the scan of heap by synchronizing blocks (each
> > parallel worker
> > scans a block and then asks for a next block to scan) among parallel
> > workers.
> > Now if we try to make sync scans work along with it, the
> > synchronization among
> > parallel workers will go for a toss. It might not be impossible to
> > make that
> > work in some way, but not sure if it is important enough for sync
> > scans to work
> > along with parallel scan.
>
> I haven't tested it, but I think it would still be helpful. The block
> accesses are still in order even during a partial scan, so why wouldn't
> it help?
>
> You might be concerned about the reporting of a block location, which
> would become more noisy with increased parallelism. But in my original
> testing, sync scans weren't very sensitive to slight deviations, because
> of caching effects.
>
> > tqueue.c is mainly designed to pass tuples between parallel workers
> > and currently it is used in Funnel operator to gather the tuples
> > generated
> > by all the parallel workers. I think we can use it for any other
> > operator
> > which needs tuple communication among parallel workers.
>
> Some specifics of the Funnel operator seem to be a part of tqueue, which
> doesn't make sense to me. For instance, reading from the set of queues
> in a round-robin fashion is part of the Funnel algorithm, and doesn't
> seem suitable for a generic tuple communication mechanism (that would
> never allow order-sensitive reading, for example).
>
>
> On Fri, Jul 3, 2015 at 10:05 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached, find the rebased version of patch.
> >
> > Note - You need to first apply the assess-parallel-safety patch which you
> > can find at:
> > http://www.postgresql.org/message-id/CAA4eK1JjsfE_dOsHTr_z1P_cBKi_X4C4X3d7Nv=VWX9fs7qdJA@mail.gmail.com
>
> I ran some performance tests on a 16 core machine with large shared
> buffers, so there is no IO involved.
> With the default value of cpu_tuple_comm_cost, parallel plan is not
> getting generated even if we are selecting 100K records from 40
> million records. So I changed the value to '0' and collected the
> performance readings.
>
> Here are the performance numbers:
>
>
> The average table row size is around 500 bytes and query selection
> column width is around 36 bytes.
> when the query selectivity goes more than 10% of total table records,
> the parallel scan performance is dropping.
>
On Tue, 2015-07-07 at 09:27 +0530, Amit Kapila wrote: > I am not sure how many blocks difference could be considered okay for > deviation? In my testing (a long time ago) deviations of tens of blocks didn't show a problem. However, an assumption of the sync scan work was that the CPU is processing faster than the IO system; whereas the parallel scan patch assumes that the IO system is faster than a single core. So perhaps the features are incompatible after all. Only testing will say for sure. Then again, syncscans are designed in such a way that they are unlikely to hurt in any situation. Even if the scans diverge (or never converge in the first place), it shouldn't be worse than starting at block zero every time. I'd prefer to leave syncscans intact for parallel scans unless you find a reasonable situation where they perform worse. This shouldn't add any complexity to the patch (if it does, let me know). Regards,Jeff Davis
Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached, find the rebased version of patch. [I haven't read this thread so far, sorry for possibly redundant comment.] I noticed that false is passed for required_outer agrument of create_partialseqscan_path(), while NULL seems to be cleaner in terms of C language. But in terms of semantics, I'm not sure this is correct anyway. Why does create_parallelscan_paths() not accept the actual rel->lateral_relids, just like create_seqscan_path() does? (See set_plain_rel_pathlist().) If there's reason for your approach, I think it's worth a comment. BTW, emacs shows whitespace on otherwise empty line parallelpath.c:57. -- Antonin Houska Cybertec Schönig & Schönig GmbH Gröhrmühlgasse 26 A-2700 Wiener Neustadt Web: http://www.postgresql-support.de, http://www.cybertec.at
>
> Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached, find the rebased version of patch.
>
> [I haven't read this thread so far, sorry for possibly redundant comment.]
>
> I noticed that false is passed for required_outer agrument of
> create_partialseqscan_path(), while NULL seems to be cleaner in terms of C
> language.
>
> But in terms of semantics, I'm not sure this is correct anyway. Why does
> create_parallelscan_paths() not accept the actual rel->lateral_relids, just
> like create_seqscan_path() does? (See set_plain_rel_pathlist().) If there's
> reason for your approach, I think it's worth a comment.
>
On Thu, Jul 16, 2015 at 1:10 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Thanks, I will fix this in next version of patch. > I am posting in this thread as I am not sure, whether it needs a separate thread or not? I gone through the code and found that the newly added funnel node is is tightly coupled with partial seq scan, in order to add many more parallel plans along with parallel seq scan, we need to remove the integration of this node with partial seq scan. To achieve the same, I have the following ideas. Plan: 1) Add the funnel path immediately for every parallel path similar to the current parallel seq scan,but during the plan generation generate the funnel plan only for the top funnel path andignore rest funnel paths. 2)Instead of adding a funnel path immediately after the partial seq scan path is generated. Add the funnel path in grouping_planner once the final rel path is generated before creating the plan. Execution: The funnel execution varies based on the below plan node. 1) partial scan - Funnel does the local scan also and returns the tuples 2) partial agg - Funnel does the merging of aggregate results and returns the final result. Any other better ideas to achieve the same? Regards, Hari Babu Fujitsu Australia
>
> On Thu, Jul 16, 2015 at 1:10 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Thanks, I will fix this in next version of patch.
> >
>
> I am posting in this thread as I am not sure, whether it needs a
> separate thread or not?
>
> I gone through the code and found that the newly added funnel node is
> is tightly coupled with
> partial seq scan, in order to add many more parallel plans along with
> parallel seq scan,
> we need to remove the integration of this node with partial seq scan.
>
> To achieve the same, I have the following ideas.
>
>
> Execution:
> The funnel execution varies based on the below plan node.
> 1) partial scan - Funnel does the local scan also and returns the tuples
> 2) partial agg - Funnel does the merging of aggregate results and
> returns the final result.
>
– Combines the results into a single tuple stream.
– Can run the child itself if no workers available.
> Any other better ideas to achieve the same?
>
On Mon, Jul 20, 2015 at 3:31 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Jul 17, 2015 at 1:22 PM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> >> On Thu, Jul 16, 2015 at 1:10 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > Thanks, I will fix this in next version of patch. >> > >> >> I am posting in this thread as I am not sure, whether it needs a >> separate thread or not? >> >> I gone through the code and found that the newly added funnel node is >> is tightly coupled with >> partial seq scan, in order to add many more parallel plans along with >> parallel seq scan, >> we need to remove the integration of this node with partial seq scan. >> > > This assumption is wrong, Funnel node can execute any node beneath > it (Refer ExecFunnel->funnel_getnext->ExecProcNode, similarly you > can see exec_parallel_stmt). Yes, funnel node can execute any node beneath it. But during the planning phase, the funnel path is added on top of partial scan path. I just want the same to enhanced to support other parallel nodes. > Yes, currently nodes supported under > Funnel nodes are limited like partialseqscan, result (due to reasons > mentioned upthread like readfuncs.s doesn't have support to read Plan > nodes which is required for worker backend to read the PlannedStmt, > ofcourse we can add them, but as we are supportting parallelism for > limited nodes, so I have not enhanced the readfuncs.c) but in general > the basic infrastructure is designed such a way that it can support > other nodes beneath it. > >> To achieve the same, I have the following ideas. >> >> >> Execution: >> The funnel execution varies based on the below plan node. >> 1) partial scan - Funnel does the local scan also and returns the tuples >> 2) partial agg - Funnel does the merging of aggregate results and >> returns the final result. >> > > Basically Funnel will execute any node beneath it, the Funnel node itself > is not responsible for doing local scan or any form of consolidation of > results, as of now, it has these 3 basic properties > – Has one child, runs multiple copies in parallel. > – Combines the results into a single tuple stream. > – Can run the child itself if no workers available. + if (!funnelstate->local_scan_done) + { + outerPlan = outerPlanState(funnelstate); + + outerTupleSlot = ExecProcNode(outerPlan); From the above code in funnel_getnext function, it directly does the calls the below node to do the scan in the backend side also. This code should refer the below node type, based on that only it can go for the backend scan. I feel executing outer plan always may not be correct for other parallel nodes. >> Any other better ideas to achieve the same? >> > > Refer slides 16-19 in Parallel Sequential Scan presentation in PGCon > https://www.pgcon.org/2015/schedule/events/785.en.html Thanks for the information. > I don't have very clear idea what is the best way to transform the nodes > in optimizer, but I think we can figure that out later unless majority > people > see that as blocking factor. I am also not finding it as a blocking factor for parallel scan. I written the above mail to get some feedback/suggestions from hackers on how to proceed in adding other parallelism nodes along with parallel scan. Regards, Hari Babu Fujitsu Australia
On Mon, Jul 6, 2015 at 8:49 PM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > I ran some performance tests on a 16 core machine with large shared > buffers, so there is no IO involved. > With the default value of cpu_tuple_comm_cost, parallel plan is not > getting generated even if we are selecting 100K records from 40 > million records. So I changed the value to '0' and collected the > performance readings. > > Here are the performance numbers: > > selectivity(millions) Seq scan(ms) Parallel scan > 2 workers > 4 workers 8 workers > 0.1 11498.93 4821.40 > 3305.84 3291.90 > 0.4 10942.98 4967.46 > 3338.58 3374.00 > 0.8 11619.44 5189.61 > 3543.86 3534.40 > 1.5 12585.51 5718.07 > 4162.71 2994.90 > 2.7 14725.66 8346.96 > 10429.05 8049.11 > 5.4 18719.00 20212.33 21815.19 > 19026.99 > 7.2 21955.79 28570.74 28217.60 > 27042.27 > > The average table row size is around 500 bytes and query selection > column width is around 36 bytes. > when the query selectivity goes more than 10% of total table records, > the parallel scan performance is dropping. Thanks for doing this testing. I think that is quite valuable. I am not too concerned about the fact that queries where more than 10% of records are selected do not speed up. Obviously, it would be nice to improve that, but I think that can be left as an area for future improvement. One thing I noticed that is a bit dismaying is that we don't get a lot of benefit from having more workers. Look at the 0.1 data. At 2 workers, if we scaled perfectly, we would be 3x faster (since the master can do work too), but we are actually 2.4x faster. Each process is on the average 80% efficient. That's respectable. At 4 workers, we would be 5x faster with perfect scaling; here we are 3.5x faster. So the third and fourth worker were about 50% efficient. Hmm, not as good. But then going up to 8 workers bought us basically nothing. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> One thing I noticed that is a bit dismaying is that we don't get a lot
> of benefit from having more workers. Look at the 0.1 data. At 2
> workers, if we scaled perfectly, we would be 3x faster (since the
> master can do work too), but we are actually 2.4x faster. Each
> process is on the average 80% efficient. That's respectable. At 4
> workers, we would be 5x faster with perfect scaling; here we are 3.5x
> faster. So the third and fourth worker were about 50% efficient.
> Hmm, not as good. But then going up to 8 workers bought us basically
> nothing.
>
Hi Amit, The latest v16 patch cannot be applied to the latest master as is. 434873806a9b1c0edd53c2a9df7c93a8ba021147 changed various lines in heapam.c, so it probably conflicts with this. [kaigai@magro sepgsql]$ cat ~/patch/parallel_seqscan_v16.patch | patch -p1 patching file src/backend/access/common/printtup.c patching file src/backend/access/heap/heapam.c Hunk #4 succeeded at 499 (offset 10 lines). Hunk #5 succeeded at 533 (offset 10 lines). Hunk #6 FAILED at 678. Hunk #7 succeeded at 790 (offset 10 lines). Hunk #8 succeeded at 821 (offset 10 lines). Hunk #9 FAILED at 955. Hunk #10 succeeded at 1365 (offset 10 lines). Hunk #11 succeeded at 1375 (offset 10 lines). Hunk #12 succeeded at 1384 (offset 10 lines). Hunk #13 succeeded at 1393 (offset 10 lines). Hunk #14 succeeded at 1402 (offset 10 lines). Hunk #15 succeeded at 1410 (offset 10 lines). Hunk #16 succeeded at 1439 (offset 10 lines). Hunk #17 succeeded at 1533 (offset 10 lines). 2 out of 17 hunks FAILED -- saving rejects to file src/backend/access/heap/heapam.c.rej: Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com> > -----Original Message----- > From: pgsql-hackers-owner@postgresql.org > [mailto:pgsql-hackers-owner@postgresql.org] On Behalf Of Amit Kapila > Sent: Thursday, July 23, 2015 8:43 PM > To: Robert Haas > Cc: Haribabu Kommi; Gavin Flower; Jeff Davis; Andres Freund; Kaigai Kouhei(海 > 外 浩平); Amit Langote; Amit Langote; Fabrízio Mello; Thom Brown; Stephen Frost; > pgsql-hackers > Subject: Re: [HACKERS] Parallel Seq Scan > > On Wed, Jul 22, 2015 at 9:14 PM, Robert Haas <robertmhaas@gmail.com> wrote: > > > > One thing I noticed that is a bit dismaying is that we don't get a lot > > of benefit from having more workers. Look at the 0.1 data. At 2 > > workers, if we scaled perfectly, we would be 3x faster (since the > > master can do work too), but we are actually 2.4x faster. Each > > process is on the average 80% efficient. That's respectable. At 4 > > workers, we would be 5x faster with perfect scaling; here we are 3.5x > > faster. So the third and fourth worker were about 50% efficient. > > Hmm, not as good. But then going up to 8 workers bought us basically > > nothing. > > > > I think the improvement also depends on how costly is the qualification, > if it is costly, even for same selectivity the gains will be shown till higher > number of clients and for simple qualifications, we will see that cost of > having more workers will start dominating (processing data over multiple > tuple queues) over the benefit we can achieve by them. > > > With Regards, > Amit Kapila. > EnterpriseDB: http://www.enterprisedb.com <http://www.enterprisedb.com/>
On Thu, Jul 23, 2015 at 9:42 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Jul 22, 2015 at 9:14 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> One thing I noticed that is a bit dismaying is that we don't get a lot >> of benefit from having more workers. Look at the 0.1 data. At 2 >> workers, if we scaled perfectly, we would be 3x faster (since the >> master can do work too), but we are actually 2.4x faster. Each >> process is on the average 80% efficient. That's respectable. At 4 >> workers, we would be 5x faster with perfect scaling; here we are 3.5x >> faster. So the third and fourth worker were about 50% efficient. >> Hmm, not as good. But then going up to 8 workers bought us basically >> nothing. >> > > I think the improvement also depends on how costly is the qualification, > if it is costly, even for same selectivity the gains will be shown till > higher > number of clients and for simple qualifications, we will see that cost of > having more workers will start dominating (processing data over multiple > tuple queues) over the benefit we can achieve by them. Yes, That's correct. when the qualification cost is increased, the performance is also increasing with number of workers. Instead of using all the configured workers per query, how about deciding number of workers based on cost of the qualification? I am not sure whether we have any information available to find out the qualification cost. This way the workers will be distributed to all backends properly. Regards, Hari Babu Fujitsu Australia
Hi Amit, Could you tell me the code intention around ExecInitFunnel()? ExecInitFunnel() calls InitFunnel() that opens the relation to be scanned by the underlying PartialSeqScan and setup ss_ScanTupleSlot of its scanstate. According to the comment of InitFunnel(), it open the relation and takes appropriate lock on it. However, an equivalent initialization is also done on InitPartialScanRelation(). Why does it acquire the relation lock twice? Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
>
> Hi Amit,
>
> Could you tell me the code intention around ExecInitFunnel()?
>
> ExecInitFunnel() calls InitFunnel() that opens the relation to be
> scanned by the underlying PartialSeqScan and setup ss_ScanTupleSlot
> of its scanstate.
> According to the comment of InitFunnel(), it open the relation and
> takes appropriate lock on it. However, an equivalent initialization
> is also done on InitPartialScanRelation().
>
> Why does it acquire the relation lock twice?
>
I think locking twice is not required, it is just that I have used the API
> On Wed, Jul 29, 2015 at 7:32 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > > > > Hi Amit, > > > > Could you tell me the code intention around ExecInitFunnel()? > > > > ExecInitFunnel() calls InitFunnel() that opens the relation to be > > scanned by the underlying PartialSeqScan and setup ss_ScanTupleSlot > > of its scanstate. > > The main need is for relation descriptor which is then required to set > the scan tuple's slot. Basically it is required for tuples flowing from > worker which will use the scan tuple slot of FunnelState. > > > According to the comment of InitFunnel(), it open the relation and > > takes appropriate lock on it. However, an equivalent initialization > > is also done on InitPartialScanRelation(). > > > > Why does it acquire the relation lock twice? > > > > I think locking twice is not required, it is just that I have used the API > ExecOpenScanRelation() which is used during other node's initialisation > due to which it lock's twice. I think in general it should be harmless. > Thanks, I could get reason of the implementation. It looks to me this design is not problematic even if Funnel gets capability to have multiple sub-plans thus is not associated with a particular relation as long as target-list and projection-info are appropriately initialized. Best regards, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
Amit, Let me ask three more detailed questions. Why Funnel has a valid qual of the subplan? The 2nd argument of make_funnel() is qualifier of the subplan (PartialSeqScan) then it is initialized at ExecInitFunnel, but never executed on the run-time. Why does Funnel node has useless qualifier expression here (even though it is harmless)? Why Funnel delivered from Scan? Even though it constructs a compatible target-list with underlying partial-scan node, it does not require the node is also delivered from Scan. For example, Sort or Append don't change the target-list definition from its input, also don't have its own qualifier. It seems to me the definition below is more suitable... typedef struct Funnel { Plan plan; int num_workers; } Funnel; Does ExecFunnel() need to have a special code path to handle EvalPlanQual()? Probably, it just calls underlying node in the local context. ExecScan() of PartialSeqScan will check its qualifier towards estate->es_epqTuple[]. Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com> > -----Original Message----- > From: pgsql-hackers-owner@postgresql.org > [mailto:pgsql-hackers-owner@postgresql.org] On Behalf Of Kouhei Kaigai > Sent: Thursday, July 30, 2015 8:45 AM > To: Amit Kapila > Cc: Robert Haas; Gavin Flower; Jeff Davis; Andres Freund; Amit Langote; Amit > Langote; Fabrízio Mello; Thom Brown; Stephen Frost; pgsql-hackers; Haribabu Kommi > Subject: Re: [HACKERS] Parallel Seq Scan > > > On Wed, Jul 29, 2015 at 7:32 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > > > > > > Hi Amit, > > > > > > Could you tell me the code intention around ExecInitFunnel()? > > > > > > ExecInitFunnel() calls InitFunnel() that opens the relation to be > > > scanned by the underlying PartialSeqScan and setup ss_ScanTupleSlot > > > of its scanstate. > > > > The main need is for relation descriptor which is then required to set > > the scan tuple's slot. Basically it is required for tuples flowing from > > worker which will use the scan tuple slot of FunnelState. > > > > > According to the comment of InitFunnel(), it open the relation and > > > takes appropriate lock on it. However, an equivalent initialization > > > is also done on InitPartialScanRelation(). > > > > > > Why does it acquire the relation lock twice? > > > > > > > I think locking twice is not required, it is just that I have used the API > > ExecOpenScanRelation() which is used during other node's initialisation > > due to which it lock's twice. I think in general it should be harmless. > > > Thanks, I could get reason of the implementation. > > It looks to me this design is not problematic even if Funnel gets capability > to have multiple sub-plans thus is not associated with a particular relation > as long as target-list and projection-info are appropriately initialized. > > Best regards, > -- > 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
>
> Amit,
>
> Let me ask three more detailed questions.
>
> Why Funnel has a valid qual of the subplan?
> The 2nd argument of make_funnel() is qualifier of the subplan
> (PartialSeqScan) then it is initialized at ExecInitFunnel,
> but never executed on the run-time. Why does Funnel node has
> useless qualifier expression here (even though it is harmless)?
>
> Why Funnel delivered from Scan? Even though it constructs
> a compatible target-list with underlying partial-scan node,
> it does not require the node is also delivered from Scan.
>
> Does ExecFunnel() need to have a special code path to handle
> EvalPlanQual()? Probably, it just calls underlying node in the
> local context. ExecScan() of PartialSeqScan will check its
> qualifier towards estate->es_epqTuple[].
>
>
> Hi Amit,
>
> The latest v16 patch cannot be applied to the latest
> master as is.
> 434873806a9b1c0edd53c2a9df7c93a8ba021147 changed various
> lines in heapam.c, so it probably conflicts with this.
>
Attachment
On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Jul 23, 2015 at 7:43 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: >> >> Hi Amit, >> >> The latest v16 patch cannot be applied to the latest >> master as is. >> 434873806a9b1c0edd53c2a9df7c93a8ba021147 changed various >> lines in heapam.c, so it probably conflicts with this. >> > > Attached, find the rebased version of patch. It fixes the comments raised > by Jeff Davis and Antonin Houska. The main changes in this version are > now it supports sync scan along with parallel sequential scan (refer > heapam.c) > and the patch has been split into two parts, first contains the code for > Funnel node and infrastructure to support the same and second contains > the code for PartialSeqScan node and its infrastructure. > Thanks for the updated patch. With subquery, parallel scan is having some problem, please refer below. postgres=# explain select * from test01 where kinkocord not in (select kinkocord from test02 where tenpocord = '001'); QUERY PLAN --------------------------------------------------------------------------------------------------Funnel on test01 (cost=0.00..155114352184.12rows=20000008 width=435) Filter: (NOT (SubPlan 1)) Number of Workers: 16 -> Partial Seq Scanon test01 (cost=0.00..155114352184.12 rows=20000008 width=435) Filter: (NOT (SubPlan 1)) SubPlan 1 -> Materialize (cost=0.00..130883.67rows=385333 width=5) -> Funnel on test02 (cost=0.00..127451.01 rows=385333 width=5) Filter: (tenpocord = '001'::bpchar) Number of Workers: 16 -> Partial Seq Scan on test02 (cost=0.00..127451.01 rows=385333 width=5) Filter: (tenpocord = '001'::bpchar) SubPlan 1 -> Materialize (cost=0.00..130883.67 rows=385333 width=5) -> Funnel on test02 (cost=0.00..127451.01 rows=385333width=5) Filter: (tenpocord = '001'::bpchar) Number of Workers: 16 -> Partial Seq Scan on test02 (cost=0.00..127451.01 rows=385333 width=5) Filter: (tenpocord = '001'::bpchar) (19 rows) postgres=# explain analyze select * from test01 where kinkocord not in (select kinkocord from test02 where tenpocord = '001'); ERROR: badly formatted node string "SUBPLAN :subLinkType 2 :testexpr"... CONTEXT: parallel worker, pid 32879 postgres=# And also regarding the number of workers (16) that is shown in the explain analyze plan are not actually allotted because the in my configuration i set the max_worker_process as 8 only. I feel the plan should show the allotted workers not the planned workers. If the query execution takes time because of lack of workers and the plan is showing as 16 workers, in that case user may think that even with 16 workers the query is slower, but actually it is not. Regards, Hari Babu Fujitsu Australia
On Wed, Sep 9, 2015 at 2:17 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > And also regarding the number of workers (16) that is shown in the > explain analyze plan are not actually allotted because the in my > configuration i set the max_worker_process as 8 only. I feel the plan > should show the allotted workers not the planned workers. > If the query execution takes time because of lack of workers and the > plan is showing as 16 workers, in that case user may think that > even with 16 workers the query is slower, but actually it is not. I would expect EXPLAIN should show the # of workers planned, and EXPLAIN ANALYZE should show both the planned and actual values. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> With subquery, parallel scan is having some problem, please refer below.
>
>
> postgres=# explain analyze select * from test01 where kinkocord not in
> (select kinkocord from test02 where tenpocord = '001');
> ERROR: badly formatted node string "SUBPLAN :subLinkType 2 :testexpr"...
> CONTEXT: parallel worker, pid 32879
> postgres=#
>
>
> On Wed, Sep 9, 2015 at 2:17 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote:
> > And also regarding the number of workers (16) that is shown in the
> > explain analyze plan are not actually allotted because the in my
> > configuration i set the max_worker_process as 8 only. I feel the plan
> > should show the allotted workers not the planned workers.
> > If the query execution takes time because of lack of workers and the
> > plan is showing as 16 workers, in that case user may think that
> > even with 16 workers the query is slower, but actually it is not.
>
> I would expect EXPLAIN should show the # of workers planned, and
> EXPLAIN ANALYZE should show both the planned and actual values.
>
On Wed, Sep 9, 2015 at 11:07 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Sep 9, 2015 at 11:47 AM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> With subquery, parallel scan is having some problem, please refer below. >> >> postgres=# explain analyze select * from test01 where kinkocord not in >> (select kinkocord from test02 where tenpocord = '001'); >> ERROR: badly formatted node string "SUBPLAN :subLinkType 2 :testexpr"... >> CONTEXT: parallel worker, pid 32879 >> postgres=# > > The problem here is that readfuncs.c doesn't have support for reading > SubPlan nodes. I have added support for some of the nodes, but it seems > SubPlan node also needs to be added. Now I think this is okay if the > SubPlan > is any node other than Funnel, but if Subplan contains Funnel, then each > worker needs to spawn other workers to execute the Subplan which I am > not sure is the best way. Another possibility could be store the results of > Subplan in some tuplestore or some other way and then pass those to workers > which again doesn't sound to be promising way considering we might have > hashed SubPlan for which we need to build a hashtable. Yet another way > could be for such cases execute the Filter in master node only. IIUC, there are two separate issues here: 1. We need to have readfuncs support for all the right plan nodes. Maybe we should just bite the bullet and add readfuncs support for all plan nodes. But if not, we can add support for whatever we need. 2. I think it's probably a good idea - at least for now, and maybe forever - to avoid nesting parallel plans inside of other parallel plans. It's hard to imagine that being a win in a case like this, and it certainly adds a lot more cases to think about. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, Sep 9, 2015 at 11:07 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Wed, Sep 9, 2015 at 11:47 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
> > wrote:
> >> With subquery, parallel scan is having some problem, please refer below.
> >>
> >> postgres=# explain analyze select * from test01 where kinkocord not in
> >> (select kinkocord from test02 where tenpocord = '001');
> >> ERROR: badly formatted node string "SUBPLAN :subLinkType 2 :testexpr"...
> >> CONTEXT: parallel worker, pid 32879
> >> postgres=#
> >
> > The problem here is that readfuncs.c doesn't have support for reading
> > SubPlan nodes. I have added support for some of the nodes, but it seems
> > SubPlan node also needs to be added. Now I think this is okay if the
> > SubPlan
> > is any node other than Funnel, but if Subplan contains Funnel, then each
> > worker needs to spawn other workers to execute the Subplan which I am
> > not sure is the best way. Another possibility could be store the results of
> > Subplan in some tuplestore or some other way and then pass those to workers
> > which again doesn't sound to be promising way considering we might have
> > hashed SubPlan for which we need to build a hashtable. Yet another way
> > could be for such cases execute the Filter in master node only.
>
> IIUC, there are two separate issues here:
>
> 1. We need to have readfuncs support for all the right plan nodes.
> Maybe we should just bite the bullet and add readfuncs support for all
> plan nodes. But if not, we can add support for whatever we need.
>
> 2. I think it's probably a good idea - at least for now, and maybe
> forever - to avoid nesting parallel plans inside of other parallel
> plans. It's hard to imagine that being a win in a case like this, and
> it certainly adds a lot more cases to think about.
>
I also think that avoiding nested parallel plans is a good step forward.
On Thu, Sep 10, 2015 at 2:12 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Sep 10, 2015 at 4:16 AM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> On Wed, Sep 9, 2015 at 11:07 AM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > On Wed, Sep 9, 2015 at 11:47 AM, Haribabu Kommi >> > <kommi.haribabu@gmail.com> >> > wrote: >> >> With subquery, parallel scan is having some problem, please refer >> >> below. >> >> >> >> postgres=# explain analyze select * from test01 where kinkocord not in >> >> (select kinkocord from test02 where tenpocord = '001'); >> >> ERROR: badly formatted node string "SUBPLAN :subLinkType 2 >> >> :testexpr"... >> >> CONTEXT: parallel worker, pid 32879 >> >> postgres=# >> > >> > The problem here is that readfuncs.c doesn't have support for reading >> > SubPlan nodes. I have added support for some of the nodes, but it seems >> > SubPlan node also needs to be added. Now I think this is okay if the >> > SubPlan >> > is any node other than Funnel, but if Subplan contains Funnel, then each >> > worker needs to spawn other workers to execute the Subplan which I am >> > not sure is the best way. Another possibility could be store the >> > results of >> > Subplan in some tuplestore or some other way and then pass those to >> > workers >> > which again doesn't sound to be promising way considering we might have >> > hashed SubPlan for which we need to build a hashtable. Yet another way >> > could be for such cases execute the Filter in master node only. >> >> IIUC, there are two separate issues here: >> > > Yes. > >> 1. We need to have readfuncs support for all the right plan nodes. >> Maybe we should just bite the bullet and add readfuncs support for all >> plan nodes. But if not, we can add support for whatever we need. >> >> 2. I think it's probably a good idea - at least for now, and maybe >> forever - to avoid nesting parallel plans inside of other parallel >> plans. It's hard to imagine that being a win in a case like this, and >> it certainly adds a lot more cases to think about. >> > > I also think that avoiding nested parallel plans is a good step forward. > I reviewed the parallel_seqscan_funnel_v17.patch and following are my comments. I will continue my review with the parallel_seqscan_partialseqscan_v17.patch. + if (inst_options) + { + instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS); + *inst_options = *instoptions; + if (inst_options) Same pointer variable check, it should be if (*inst_options) as per the estimate and store functions. + if (funnelstate->ss.ps.ps_ProjInfo) + slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot; + else + slot = funnelstate->ss.ss_ScanTupleSlot; Currently, there will not be a projinfo for funnel node. So always it uses the scan tuple slot. In case if it is different, we need to add the ExecProject call in ExecFunnel function. Currently it is not present, either we can document it or add the function call. + if (!((*dest->receiveSlot) (slot, dest))) + break; and +void +TupleQueueFunnelShutdown(TupleQueueFunnel *funnel) +{ + if (funnel) + { + int i; + shm_mq_handle *mqh; + shm_mq *mq; + for (i = 0; i < funnel->nqueues; i++) + { + mqh = funnel->queue[i]; + mq = shm_mq_get_queue(mqh); + shm_mq_detach(mq); + } + } +} Using this function, the backend detaches from the message queue, so that the workers which are trying to put results into the queues gets an error message as SHM_MQ_DETACHED. Then worker finshes the execution of the plan. For this reason all the printtup return types are changed from void to bool. But this way the worker doesn't get exited until it tries to put a tuple in the queue. If there are no valid tuples that satisfy the condition, then it may take time for the workers to exit. Am I correct? I am not sure how frequent such scenarios can occur. + if (parallel_seqscan_degree >= MaxConnections) + { + write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname); + ExitPostmaster(1); + } The error condition works only during server start. User still can set parallel seqscan degree more than max connection at super user session level and etc. + if (!parallelstmt->inst_options) + (*receiver->rDestroy) (receiver); Why only when there is no instruementation only, the receiver needs to be destroyed? Regards, Hari Babu Fujitsu Australia
On Thu, Sep 10, 2015 at 12:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> 2. I think it's probably a good idea - at least for now, and maybe >> forever - to avoid nesting parallel plans inside of other parallel >> plans. It's hard to imagine that being a win in a case like this, and >> it certainly adds a lot more cases to think about. > > I also think that avoiding nested parallel plans is a good step forward. Doing that as a part of the assess parallel safety patch was trivial, so I did. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached, find the rebased version of patch. It fixes the comments raised > by Jeff Davis and Antonin Houska. The main changes in this version are > now it supports sync scan along with parallel sequential scan (refer > heapam.c) > and the patch has been split into two parts, first contains the code for > Funnel node and infrastructure to support the same and second contains > the code for PartialSeqScan node and its infrastructure. + if (es->analyze && nodeTag(plan) == T_Funnel) Why not IsA()? + FinishParallelSetupAndAccumStats((FunnelState *)planstate); Shouldn't there be a space before planstate? + /* inform executor to collect buffer usage stats from parallel workers. */ + estate->total_time = queryDesc->totaltime ? 1 : 0; Boy, the comment sure doesn't seem to match the code. + * Accumulate the stats by parallel workers before stopping the + * node. Suggest: "Accumulate stats from parallel workers before stopping node". + * If we are not able to send the tuple, then we assume that + * destination has closed and we won't be able to send any more + * tuples so we just end the loop. Suggest: "If we are not able to send the tuple, we assume the destination has closed and no more tuples can be sent. If that's the case, end the loop." +static void +EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params, + List *serialized_param_exec_vals, + int instOptions, Size *params_size, + Size *params_exec_size); +static void +StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params, + List *serialized_param_exec_vals, + int instOptions, Size params_size, + Size params_exec_size, + char **inst_options_space, + char **buffer_usage_space); Whitespace doesn't look like PostgreSQL style. Maybe run pgindent on the newly-added files? +/* + * This is required for parallel plan execution to fetch the information + * from dsm. + */ This comment doesn't really say anything. Can we get a better one? + /* + * We expect each worker to populate the BufferUsage structure + * allocated by master backend and then master backend will aggregate + * all the usage along with it's own, so account it for each worker. + */ This also needs improvement. Especially because... + /* + * We expect each worker to populate the instrumentation structure + * allocated by master backend and then master backend will aggregate + * all the information, so account it for each worker. + */ ...it's almost identical to this one. + * Store bind parameter's list in dynamic shared memory. This is + * used for parameters in prepared query. s/bind parameter's list/bind parameters/. I think you could drop the second sentence, too. + /* + * Store PARAM_EXEC parameters list in dynamic shared memory. This is + * used for evaluation plan->initPlan params. + */ So is the previous block for PARAM_EXTERN and this is PARAM_EXEC? If so, maybe that could be more clearly laid out. +GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params, Could this be a static function? Will it really be needed outside this file? And is there any use case for letting some of the arguments be NULL? Seems kind of an awkward API. +bool +ExecParallelBufferUsageAccum(Node *node) +{ + if (node == NULL) + return false; + + switch (nodeTag(node)) + { + case T_FunnelState: + { + FinishParallelSetupAndAccumStats((FunnelState*)node); + return true; + } + break; + default: + break; + } + + (void) planstate_tree_walker((Node*)((PlanState *)node)->lefttree, NULL, + ExecParallelBufferUsageAccum, 0); + (void) planstate_tree_walker((Node*)((PlanState *)node)->righttree, NULL, + ExecParallelBufferUsageAccum, 0); + return false; +} This seems wacky. I mean, isn't the point of planstate_tree_walker() that the callback itself doesn't have to handle recursion like this? And if not, then this wouldn't be adequate anyway, because some planstate nodes have children that are not in lefttree or righttree (cf. explain.c). + currentRelation = ExecOpenScanRelation(estate, + ((SeqScan *) node->ss.ps.plan)->scanrelid, + eflags); I can't see how this can possibly be remotely correct. The funnel node shouldn't be limited to scanning a baserel (cf. fdw_scan_tlist). +void ExecAccumulateInstInfo(FunnelState *node) Another place where pgindent would help. There are a bunch of others I noticed too, but I'm just mentioning a few here to make the point. + buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage))); Cast it to a BufferUsage * first. Then you can use &foo[i] to find the i'th element. + /* + * Re-initialize the parallel context and workers to perform + * rescan of relation. We want to gracefully shutdown all the + * workers so that they should be able to propagate any error + * or other information to master backend before dying. + */ + FinishParallelSetupAndAccumStats(node); Somehow, this makes me feel like that function is badly named. +/* + * _readPlanInvalItem + */ +static PlanInvalItem * +_readPlanInvalItem(void) +{ + READ_LOCALS(PlanInvalItem); + + READ_INT_FIELD(cacheId); + READ_UINT_FIELD(hashValue); + + READ_DONE(); +} I don't see why we should need to be able to copy PlanInvalItems. In fact, it seems like a bad idea. +#parallel_setup_cost = 0.0 # same scale as above +#define DEFAULT_PARALLEL_SETUP_COST 0.0 This value is probably a bit on the low side. +int parallel_seqscan_degree = 0; I think we should have a GUC for the maximum degree of parallelism in a query generally, not the maximum degree of parallel sequential scan. + if (parallel_seqscan_degree >= MaxConnections) + { + write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname); + ExitPostmaster(1); + } I think this check is thoroughly unnecessary. It's comparing to the wrong thing anyway, because what actually matters is max_worker_processes, not max_connections. But in any case there is no need for the check. If somebody stupidly tries an unreasonable value for the maximum degree of parallelism, they won't get that many workers, but nothing will break. It's no worse than setting any other query planner costing parameter to an insane value. --- a/src/include/access/heapam.h +++ b/src/include/access/heapam.h @@ -126,6 +126,7 @@ extern void heap_rescan_set_params(HeapScanDesc scan, ScanKey key,extern void heap_endscan(HeapScanDesc scan);extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirectiondirection); +extern bool heap_fetch(Relation relation, Snapshot snapshot, Stray whitespace change. More later, that's what I noticed on a first read through. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Jul 22, 2015 at 10:44 AM, Robert Haas <robertmhaas@gmail.com> wrote: > One thing I noticed that is a bit dismaying is that we don't get a lot > of benefit from having more workers. Look at the 0.1 data. At 2 > workers, if we scaled perfectly, we would be 3x faster (since the > master can do work too), but we are actually 2.4x faster. Each > process is on the average 80% efficient. That's respectable. At 4 > workers, we would be 5x faster with perfect scaling; here we are 3.5x > faster. So the third and fourth worker were about 50% efficient. > Hmm, not as good. But then going up to 8 workers bought us basically > nothing. ...sorry for bumping up this mail from July... I don't think you meant to imply it, but why should we be able to scale perfectly? Even when the table fits entirely in shared_buffers, I would expect memory bandwidth to become the bottleneck before a large number of workers are added. Context switching might also be problematic. I have almost no sense of whether this is below or above par, which is what I'm really curious about. FWIW, I think that parallel sort will scale somewhat better. -- Peter Geoghegan
On Thu, Sep 17, 2015 at 6:10 AM, Robert Haas <robertmhaas@gmail.com> wrote: > On Thu, Sep 10, 2015 at 12:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >>> 2. I think it's probably a good idea - at least for now, and maybe >>> forever - to avoid nesting parallel plans inside of other parallel >>> plans. It's hard to imagine that being a win in a case like this, and >>> it certainly adds a lot more cases to think about. >> >> I also think that avoiding nested parallel plans is a good step forward. > > Doing that as a part of the assess parallel safety patch was trivial, so I did. > I tried with latest HEAD code, seems to be problem is present in other scenarios. postgres=# explain select * from tbl a where exists (select 1 from tbl b where a.f1=b.f1 limit 0); QUERY PLAN --------------------------------------------------------------------------------------Funnel on tbl a (cost=0.00..397728310227.27rows=5000000 width=214) Filter: (SubPlan 1) Number of Workers: 10 -> Partial Seq Scan on tbla (cost=0.00..397727310227.27 rows=5000000 width=214) Filter: (SubPlan 1) SubPlan 1 -> Limit (cost=0.00..437500.00 rows=1 width=0) -> Seq Scan on tbl b (cost=0.00..437500.00 rows=1 width=0) Filter: (a.f1 =f1) SubPlan 1 -> Limit (cost=0.00..437500.00 rows=1 width=0) -> Seq Scan on tbl b (cost=0.00..437500.00rows=1 width=0) Filter: (a.f1 = f1) (13 rows) postgres=# explain analyze select * from tbl a where exists (select 1 from tbl b where a.f1=b.f1 limit 0); ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... LOG: worker process: parallel worker for PID 8775 (PID 9121) exited with exit code 1 ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... LOG: worker process: parallel worker for PID 8775 (PID 9116) exited with exit code 1 LOG: worker process: parallel worker for PID 8775 (PID 9119) exited with exit code 1 ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... LOG: worker process: parallel worker for PID 8775 (PID 9117) exited with exit code 1 LOG: worker process: parallel worker for PID 8775 (PID 9114) exited with exit code 1 ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... LOG: worker process: parallel worker for PID 8775 (PID 9118) exited with exit code 1 ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... CONTEXT: parallel worker, pid 9115 STATEMENT: explain analyze select * from tbl a where exists (select 1 from tbl b where a.f1=b.f1 limit 0); LOG: worker process: parallel worker for PID 8775 (PID 9115) exited with exit code 1 LOG: worker process: parallel worker for PID 8775 (PID 9120) exited with exit code 1 ERROR: badly formatted node string "SUBPLAN :subLinkType 0 :testexpr"... CONTEXT: parallel worker, pid 9115 Regards, Hari Babu Fujitsu Australia
On Mon, Sep 14, 2015 at 11:04 PM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > Using this function, the backend detaches from the message queue, so > that the workers > which are trying to put results into the queues gets an error message > as SHM_MQ_DETACHED. > Then worker finshes the execution of the plan. For this reason all the > printtup return > types are changed from void to bool. > > But this way the worker doesn't get exited until it tries to put a > tuple in the queue. > If there are no valid tuples that satisfy the condition, then it may > take time for the workers > to exit. Am I correct? I am not sure how frequent such scenarios can occur. Yes, that's a problem. It's probably not that bad as long as the only thing that can occur under a Funnel node is a sequential scan, although even then the filter condition on the sequential scan could be something expensive or highly selective. But it will get a lot worse when we get the ability to push joins below the funnel. I welcome ideas for solving this problem. Basically, the problem is that we may need to shut down the executor before execution is complete. This can happen because we're beneath a limit node; it can also happen because we're on the inner side of a semijoin and have already found one match. Presumably, parallel plans in such case will be rare. But there may be cases where they happen, and so we need some way to handle it. One idea is that the workers could exit by throwing an ERROR, maybe after setting some flag first to say, hey, this isn't a *real* error, we're just doing this to achieve a non-local transfer of control. But then we need to make sure that any instrumentation statistics still get handled properly, which is maybe not so easy. And it seems like there might be other problems with things not getting shut down properly as well. Any code that expects a non-local exit to lead to a (sub)transaction abort potentially gets broken by this approach. Another idea is to try to gradually enrich the set of places that check for shutdown. So for example at the beginning of ExecProcNode() we could add a check at the beginning to return NULL if the flag's been set; that would probably dampen the amount of additional work that could get done in many common scenarios. But that might break a bunch of things too, and it's far from a complete solution anyway: for example, we could be stuck down inside some user-defined function, and I don't see that there's much choice in that case to run the function to conclusion. This problem essentially happens because we're hoping that the workers in parallel mode will "run ahead" of the master, producing tuples for it to read before it gets to the point of sitting and waiting for them. Indeed, if that happens, we've missed the boat entirely. But then that opens up the problem that the master could always decide it doesn't need any tuples after all. Anyone have a smart idea for how to attack this? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > [ new patches ] + pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN); This is total nonsense. You can't hard-code the key that's used for the scan, because we need to be able to support more than one parallel operator beneath the same funnel. For example: Append -> Partial Seq Scan -> Partial Seq Scan Each partial sequential scan needs to have a *separate* key, which will need to be stored in either the Plan or the PlanState or both (not sure exactly). Each partial seq scan needs to get assigned a unique key there in the master, probably starting from 0 or 100 or something and counting up, and then this code needs to extract that value and use it to look up the correct data for that scan. + case T_ResultState: + { + PlanState *planstate = ((ResultState*)node)->ps.lefttree; + + return planstate_tree_walker((Node*)planstate, pcxt, + ExecParallelInitializeDSM, pscan_size); + } This looks like another instance of using the walker incorrectly. Nodes where you just want to let the walk continue shouldn't need to be enumerated; dispatching like this should be the default case. + case T_Result: + fix_opfuncids((Node*) (((Result *)node)->resconstantqual)); + break; Seems similarly wrong. + * cost_patialseqscan Typo. The actual function name has the same typo. + num_parallel_workers = parallel_seqscan_degree; + if (parallel_seqscan_degree <= estimated_parallel_workers) + num_parallel_workers = parallel_seqscan_degree; + else + num_parallel_workers = estimated_parallel_workers; Use Min? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Sep 17, 2015 at 6:10 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> > On Thu, Sep 10, 2015 at 12:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >>> 2. I think it's probably a good idea - at least for now, and maybe
> >>> forever - to avoid nesting parallel plans inside of other parallel
> >>> plans. It's hard to imagine that being a win in a case like this, and
> >>> it certainly adds a lot more cases to think about.
> >>
> >> I also think that avoiding nested parallel plans is a good step forward.
> >
> > Doing that as a part of the assess parallel safety patch was trivial, so I did.
> >
>
> I tried with latest HEAD code, seems to be problem is present in other
> scenarios.
>
With Regards,
Amit Kapila.
On Thu, Sep 17, 2015 at 12:03 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > As mentioned previously [1], we have to do two different things to make > this work, Robert seems to have taken care of one of those (basically > second point in mail[1]) and still another one needs to be taken care > which is to provide support of reading subplans in readfuncs.c and that > will solve the problem you are seeing now. Thanks for the information. During my test, I saw a plan change from parallel seq scan to seq scan for the first reported query. So I thought that all scenarios are corrected as not to generate the parallel seq scan. Regards, Hari Babu Fujitsu Australia
>
> On Thu, Sep 10, 2015 at 12:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> 2. I think it's probably a good idea - at least for now, and maybe
> >> forever - to avoid nesting parallel plans inside of other parallel
> >> plans. It's hard to imagine that being a win in a case like this, and
> >> it certainly adds a lot more cases to think about.
> >
> > I also think that avoiding nested parallel plans is a good step forward.
>
> Doing that as a part of the assess parallel safety patch was trivial, so I did.
>
+ * a parallel worker. We might eventually be able to relax this
+ * restriction, but for now it seems best not to have parallel workers
+ * trying to create their own parallel workers.
+ */
+ glob->parallelModeOK = (cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 &&
+ IsUnderPostmaster && dynamic_shared_memory_type != DSM_IMPL_NONE &&
+ parse->commandType == CMD_SELECT && !parse->hasModifyingCTE &&
+ parse->utilityStmt == NULL && !IsParallelWorker() &&
+ !contain_parallel_unsafe((Node *) parse);
>
> >
>
> I reviewed the parallel_seqscan_funnel_v17.patch and following are my comments.
> I will continue my review with the parallel_seqscan_partialseqscan_v17.patch.
>
> + if (inst_options)
> + {
> + instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
> + *inst_options = *instoptions;
> + if (inst_options)
>
> Same pointer variable check, it should be if (*inst_options) as per the
> estimate and store functions.
>
>
> + if (funnelstate->ss.ps.ps_ProjInfo)
> + slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
> + else
> + slot = funnelstate->ss.ss_ScanTupleSlot;
>
> Currently, there will not be a projinfo for funnel node.
> the scan tuple slot. In case if it is different, we need to add the ExecProject
> call in ExecFunnel function.
>
>
> + if (!((*dest->receiveSlot) (slot, dest)))
> + break;
>
> and
>
> +void
> +TupleQueueFunnelShutdown(TupleQueueFunnel *funnel)
> +{
> + if (funnel)
> + {
> + int i;
> + shm_mq_handle *mqh;
> + shm_mq *mq;
> + for (i = 0; i < funnel->nqueues; i++)
> + {
> + mqh = funnel->queue[i];
> + mq = shm_mq_get_queue(mqh);
> + shm_mq_detach(mq);
> + }
> + }
> +}
>
>
> Using this function, the backend detaches from the message queue, so
> that the workers
> which are trying to put results into the queues gets an error message
> as SHM_MQ_DETACHED.
> Then worker finshes the execution of the plan. For this reason all the
> printtup return
> types are changed from void to bool.
>
> But this way the worker doesn't get exited until it tries to put a
> tuple in the queue.
> If there are no valid tuples that satisfy the condition, then it may
> take time for the workers
> to exit. Am I correct? I am not sure how frequent such scenarios can occur.
>
>
> + if (parallel_seqscan_degree >= MaxConnections)
> + {
> + write_stderr("%s: parallel_scan_degree must be less than
> max_connections\n", progname);
> + ExitPostmaster(1);
> + }
>
> The error condition works only during server start. User still can set
> parallel seqscan degree
> more than max connection at super user session level and etc.
>
>
> + if (!parallelstmt->inst_options)
> + (*receiver->rDestroy) (receiver);
>
> Why only when there is no instruementation only, the receiver needs to
> be destroyed?
>
On Thu, Sep 17, 2015 at 2:54 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > As per my understanding, what you have done there will not prohibit such > cases. > > + * For now, we don't try to use parallel mode if we're running inside > + * a parallel worker. We might eventually be able to relax this > + * restriction, but for now it seems best not to have parallel workers > + * trying to create their own parallel workers. > + */ > + glob->parallelModeOK = (cursorOptions & CURSOR_OPT_PARALLEL_OK) != 0 && > + IsUnderPostmaster && dynamic_shared_memory_type != DSM_IMPL_NONE && > + parse->commandType == CMD_SELECT && !parse->hasModifyingCTE && > + parse->utilityStmt == NULL && !IsParallelWorker() && > + !contain_parallel_unsafe((Node *) parse); > > > IIUC, your are referring to !IsParallelWorker() check in above code. If > yes, > then I think it won't work because we generate the plan in master backend, > parallel worker will never exercise this code. I have tested it as well > with > below example and it still generates SubPlan as Funnel. You're right. That's still a good check, because some function called in the worker might try to execute a query all of its own, but it doesn't prevent the case you are talking about. > Here the subplan is generated before the top level plan and while generation > of subplan we can't predict whether it is okay to generate it as Funnel or > not, > because it might be that top level plan is non-Funnel. Also if such a > subplan > is actually an InitPlan, then we are safe (as we execute the InitPlans in > master backend and then pass the result to parallel worker) even if top > level > plan is Funnel. I think the place where we can catch this is during the > generation of Funnel path, basically we can evaluate if any nodes beneath > Funnel node has 'filter' or 'targetlist' as another Funnel node, then we > have > two options to proceed: > a. Mark such a filter or target list as non-pushable which will indicate > that > they need to be executed only in master backend. If we go with this > option, then we have to make Funnel node capable of evaluating Filter > and Targetlist which is not a big thing. > b. Don't choose the current path as Funnel path. > > I prefer second one as that seems to be simpler as compare to first and > there doesn't seem to be much benefit in going by first. > > Any better ideas? I haven't studied the planner logic in enough detail yet to have a clear opinion on this. But what I do think is that this is a very good reason why we should bite the bullet and add outfuncs/readfuncs support for all Plan nodes. Otherwise, we're going to have to scan subplans for nodes we're not expecting to see there, which seems silly. We eventually want to allow all of those nodes in the worker anyway. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ new patches ]
>
> + pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
>
> This is total nonsense. You can't hard-code the key that's used for
> the scan, because we need to be able to support more than one parallel
> operator beneath the same funnel. For example:
>
> Append
> -> Partial Seq Scan
> -> Partial Seq Scan
>
> Each partial sequential scan needs to have a *separate* key, which
> will need to be stored in either the Plan or the PlanState or both
> (not sure exactly). Each partial seq scan needs to get assigned a
> unique key there in the master, probably starting from 0 or 100 or
> something and counting up, and then this code needs to extract that
> value and use it to look up the correct data for that scan.
>
On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > > Attached, find the rebased version of patch. Here are the performance test results: Query selectivity HashAgg HashAgg (million) + seqscan(ms) + parallel seq scan(ms) 2 workers 4 workers 8 workers $1 <= '001' 0.1 16717.00 7086.00 4459.00 2912.00 $1 <= '004' 0.4 17962.00 7410.00 4651.00 2977.00 $1 <= '008' 0.8 18870.00 7849.00 4868.00 3092.00 $1 <= '016' 1.5 21368.00 8645.00 6800.00 3486.00 $1 <= '030' 2.7 24622.00 14796.00 13108.00 9981.00 $1 <= '060' 5.4 31690.00 29839.00 26544.00 23814.00 $1 <= '080' 7.2 37147.00 40485.00 35763.00 32679.00 Table Size - 18GB Total rows - 40 million Configuration: Shared_buffers - 12GB max_wal_size - 5GB checkpoint_timeout - 15min work_mem - 1GB System: CPU - 16 core RAM - 64GB Query: SELECT col1, col2, SUM(col3) AS sum_col3, SUM(col4) AS sum_col4, SUM(col5) AS sum_col5, SUM(col6) AS sum_col6 FROM public.test01 WHERE col1 <= $1 AND col7 = '01' AND col8 = '0' GROUP BY col2,col1; And also attached perf results for selectivity of 0.1 million and 5.4 million cases for analysis. Regards, Hari Babu Fujitsu Australia
Attachment
>
> On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> + /*
> + * We expect each worker to populate the BufferUsage structure
> + * allocated by master backend and then master backend will aggregate
> + * all the usage along with it's own, so account it for each worker.
> + */
>
> This also needs improvement. Especially because...
>
> + /*
> + * We expect each worker to populate the instrumentation structure
> + * allocated by master backend and then master backend will aggregate
> + * all the information, so account it for each worker.
> + */
>
> ...it's almost identical to this one.
>
>
> +GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
>
> Could this be a static function? Will it really be needed outside this file?
>
> And is there any use case for letting some of the arguments be NULL?
>
> +bool
> +ExecParallelBufferUsageAccum(Node *node)
> +{
> + if (node == NULL)
> + return false;
> +
> + switch (nodeTag(node))
> + {
> + case T_FunnelState:
> + {
> + FinishParallelSetupAndAccumStats((FunnelState*)node);
> + return true;
> + }
> + break;
> + default:
> + break;
> + }
> +
> + (void) planstate_tree_walker((Node*)((PlanState *)node)->lefttree, NULL,
> + ExecParallelBufferUsageAccum, 0);
> + (void) planstate_tree_walker((Node*)((PlanState *)node)->righttree, NULL,
> + ExecParallelBufferUsageAccum, 0);
> + return false;
> +}
>
> This seems wacky. I mean, isn't the point of planstate_tree_walker()
> that the callback itself doesn't have to handle recursion like this?
> And if not, then this wouldn't be adequate anyway, because some
> planstate nodes have children that are not in lefttree or righttree
> (cf. explain.c).
>
> + currentRelation = ExecOpenScanRelation(estate,
> + ((SeqScan *)
> node->ss.ps.plan)->scanrelid,
> + eflags);
>
> I can't see how this can possibly be remotely correct. The funnel
> node shouldn't be limited to scanning a baserel (cf. fdw_scan_tlist).
>
>
> + buffer_usage_worker = (BufferUsage *)(buffer_usage + (i *
> sizeof(BufferUsage)));
>
> Cast it to a BufferUsage * first. Then you can use &foo[i] to find
> the i'th element.
>
> + /*
> + * Re-initialize the parallel context and workers to perform
> + * rescan of relation. We want to gracefully shutdown all the
> + * workers so that they should be able to propagate any error
> + * or other information to master backend before dying.
> + */
> + FinishParallelSetupAndAccumStats(node);
>
> Somehow, this makes me feel like that function is badly named.
>
> +/*
> + * _readPlanInvalItem
> + */
> +static PlanInvalItem *
> +_readPlanInvalItem(void)
> +{
> + READ_LOCALS(PlanInvalItem);
> +
> + READ_INT_FIELD(cacheId);
> + READ_UINT_FIELD(hashValue);
> +
> + READ_DONE();
> +}
>
> I don't see why we should need to be able to copy PlanInvalItems. In
> fact, it seems like a bad idea.
>
> +#parallel_setup_cost = 0.0 # same scale as above
> +#define DEFAULT_PARALLEL_SETUP_COST 0.0
>
> This value is probably a bit on the low side.
>
On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> Attached, find the rebased version of patch.
Here are the performance test results:
Query selectivity HashAgg HashAgg
(million) + seqscan(ms) +
parallel seq scan(ms)
2
workers 4 workers 8 workers
$1 <= '001' 0.1 16717.00 7086.00
4459.00 2912.00
$1 <= '004' 0.4 17962.00 7410.00
4651.00 2977.00
$1 <= '008' 0.8 18870.00 7849.00
4868.00 3092.00
$1 <= '016' 1.5 21368.00 8645.00
6800.00 3486.00
$1 <= '030' 2.7 24622.00 14796.00 13108.00
9981.00
$1 <= '060' 5.4 31690.00 29839.00 26544.00
23814.00
$1 <= '080' 7.2 37147.00 40485.00 35763.00
32679.00
And also attached perf results for selectivity of 0.1 million and 5.4
million cases for analysis.
Amit Kapila.
> On Thu, Sep 17, 2015 at 2:54 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
>
> > Here the subplan is generated before the top level plan and while generation
> > of subplan we can't predict whether it is okay to generate it as Funnel or
> > not,
> > because it might be that top level plan is non-Funnel. Also if such a
> > subplan
> > is actually an InitPlan, then we are safe (as we execute the InitPlans in
> > master backend and then pass the result to parallel worker) even if top
> > level
> > plan is Funnel. I think the place where we can catch this is during the
> > generation of Funnel path, basically we can evaluate if any nodes beneath
> > Funnel node has 'filter' or 'targetlist' as another Funnel node, then we
> > have
> > two options to proceed:
> > a. Mark such a filter or target list as non-pushable which will indicate
> > that
> > they need to be executed only in master backend. If we go with this
> > option, then we have to make Funnel node capable of evaluating Filter
> > and Targetlist which is not a big thing.
> > b. Don't choose the current path as Funnel path.
> >
> > I prefer second one as that seems to be simpler as compare to first and
> > there doesn't seem to be much benefit in going by first.
> >
> > Any better ideas?
>
> I haven't studied the planner logic in enough detail yet to have a
> clear opinion on this. But what I do think is that this is a very
> good reason why we should bite the bullet and add outfuncs/readfuncs
> support for all Plan nodes. Otherwise, we're going to have to scan
> subplans for nodes we're not expecting to see there, which seems
> silly. We eventually want to allow all of those nodes in the worker
> anyway.
>
makes sense to me. There are 39 plan nodes and it seems we have
On Fri, Sep 18, 2015 at 4:03 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> >> Attached, find the rebased version of patch. > > Here are the performance test results: Thanks, this is really interesting. I'm very surprised by how much kernel overhead this shows. I wonder where that's coming from. The writes to and reads from the shm_mq shouldn't need to touch the kernel at all except for page faults; that's why I chose this form of IPC. It could be that the signals which are sent for flow control are chewing up a lot of cycles, but if that's the problem, it's not very clear from here. copy_user_generic_string doesn't sound like something related to signals. And why all the kernel time in _spin_lock? Maybe perf -g would help us tease out where this kernel time is coming from. Some of this may be due to rapid context switching. Suppose the master process is the bottleneck. Then each worker will fill up the queue and go to sleep. When the master reads a tuple, the worker has to wake up and write a tuple, and then it goes back to sleep. This might be an indication that we need a bigger shm_mq size. I think that would be experimenting with: if we double or quadruple or increase by 10x the queue size, what happens to performance? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 17, 2015 at 11:44 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Okay, but I think the same can be achieved with this as well. Basic idea > is that each worker will work on one planned statement at a time and in > above case there will be two different planned statements and they will > store partial seq scan related information in two different loctions in > toc, although the key (PARALLEL_KEY_SCAN) would be same and I think this > will quite similar to what we are already doing for response queues. > The worker will work on one of those keys based on planned statement > which it chooses to execute. I have explained this in somewhat more details > in one of my previous mails [1]. shm_toc keys are supposed to be unique. If you added more than one with the same key, there would be no look up the second one. That was intentional, and I don't want to revise it. I don't want to have multiple PlannedStmt objects in any case. That doesn't seem like the right approach. I think passing down an Append tree with multiple Partial Seq Scan children to be run in order is simple and clear, and I don't see why we would do it any other way. The master should be able to generate a plan and then copy the part of it below the Funnel and send it to the worker. But there's clearly never more than one PlannedStmt in the master, so where would the other ones come from in the worker? There's no reason to introduce that complexity. >> Each partial sequential scan needs to have a *separate* key, which >> will need to be stored in either the Plan or the PlanState or both >> (not sure exactly). Each partial seq scan needs to get assigned a >> unique key there in the master, probably starting from 0 or 100 or >> something and counting up, and then this code needs to extract that >> value and use it to look up the correct data for that scan. > > In that case also, multiple workers can worker on same key, assuming > in your above example, multiple workers will be required to execute > each partial seq scan. In this case we might need to see how to map > instrumentation information for a particular execution. That was discussed on the nearby thread about numbering plan nodes. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Sep 18, 2015 at 6:55 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> + currentRelation = ExecOpenScanRelation(estate, >> + ((SeqScan *) >> node->ss.ps.plan)->scanrelid, >> + eflags); >> >> I can't see how this can possibly be remotely correct. The funnel >> node shouldn't be limited to scanning a baserel (cf. fdw_scan_tlist). >> > > This is mainly used for generating tuple descriptor and that tuple > descriptor will be used for forming scanslot, funnel node itself won't > do any scan. However, we can completely eliminate this InitFunnel() > function and use ExecAssignProjectionInfo() instead of > ExecAssignScanProjectionInfo() to form the projection info. That sounds like a promising approach. >> + buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * >> sizeof(BufferUsage))); >> >> Cast it to a BufferUsage * first. Then you can use &foo[i] to find >> the i'th element. > > Do you mean to say that the way code is written won't work? > Values of structure BufferUsage for each worker is copied into string > buffer_usage which I believe could be fetched in above way. I'm just complaining about the style. If bar is a char*, then these are all equivalent: foo = (Quux *) (bar + (i * sizeof(Quux)); foo = ((Quux *) bar) + i; foo = &((Quux *) bar)[i]; baz = (Quux *) bar; foo = &baz[i]; >> + /* >> + * Re-initialize the parallel context and workers to perform >> + * rescan of relation. We want to gracefully shutdown all the >> + * workers so that they should be able to propagate any error >> + * or other information to master backend before dying. >> + */ >> + FinishParallelSetupAndAccumStats(node); >> >> Somehow, this makes me feel like that function is badly named. > > I think here comment seems to be slightly misleading, shall we > change the comment as below: > > Destroy the parallel context to gracefully shutdown all the > workers so that they should be able to propagate any error > or other information to master backend before dying. Well, why does a function that destroys the parallel context have a name that starts with FinishParallelSetup? It sounds like it is tearing things down, not finishing setup. >> +#parallel_setup_cost = 0.0 # same scale as above >> +#define DEFAULT_PARALLEL_SETUP_COST 0.0 >> >> This value is probably a bit on the low side. > > How about keeping it as 10.0? Really? I would have guessed that the correct value was in the tens of thousands. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Sep 18, 2015 at 12:56 PM, Robert Haas <robertmhaas@gmail.com> wrote: > On Thu, Sep 17, 2015 at 11:44 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Okay, but I think the same can be achieved with this as well. Basic idea >> is that each worker will work on one planned statement at a time and in >> above case there will be two different planned statements and they will >> store partial seq scan related information in two different loctions in >> toc, although the key (PARALLEL_KEY_SCAN) would be same and I think this >> will quite similar to what we are already doing for response queues. >> The worker will work on one of those keys based on planned statement >> which it chooses to execute. I have explained this in somewhat more details >> in one of my previous mails [1]. > > shm_toc keys are supposed to be unique. If you added more than one > with the same key, there would be no look up the second one. That was > intentional, and I don't want to revise it. > > I don't want to have multiple PlannedStmt objects in any case. That > doesn't seem like the right approach. I think passing down an Append > tree with multiple Partial Seq Scan children to be run in order is > simple and clear, and I don't see why we would do it any other way. > The master should be able to generate a plan and then copy the part of > it below the Funnel and send it to the worker. But there's clearly > never more than one PlannedStmt in the master, so where would the > other ones come from in the worker? There's no reason to introduce > that complexity. Also, as KaiGai pointed out on the other thread, even if you DID pass two PlannedStmt nodes to the worker, you still need to know which one goes with which ParallelHeapScanDesc. If both of the ParallelHeapScanDesc nodes are stored under the same key, then you can't do that. That's why, as discussed in the other thread, we need some way of uniquely identifying a plan node. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Sep 18, 2015 at 9:45 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Sep 18, 2015 at 1:33 PM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> >> On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > >> > Attached, find the rebased version of patch. >> >> Here are the performance test results: >> >> Query selectivity HashAgg HashAgg >> (million) + seqscan(ms) + >> parallel seq scan(ms) >> 2 >> workers 4 workers 8 workers >> $1 <= '001' 0.1 16717.00 7086.00 >> 4459.00 2912.00 >> $1 <= '004' 0.4 17962.00 7410.00 >> 4651.00 2977.00 >> $1 <= '008' 0.8 18870.00 7849.00 >> 4868.00 3092.00 >> $1 <= '016' 1.5 21368.00 8645.00 >> 6800.00 3486.00 >> $1 <= '030' 2.7 24622.00 14796.00 13108.00 >> 9981.00 >> $1 <= '060' 5.4 31690.00 29839.00 26544.00 >> 23814.00 >> $1 <= '080' 7.2 37147.00 40485.00 35763.00 >> 32679.00 >> > > I think here probably when the selectivity is more than 5, then it should > not have selected Funnel plan. Have you by any chance changed > cpu_tuple_comm_cost? If not, then you can try by setting value of > parallel_setup_cost (may be 10) and then see if it selects the Funnel > Plan. Is it possible for you to check the cost difference of Sequence > and Funnel plan, hopefully explain or explain analyze should be sufficient? Yes, I changed cpu_tuple_comm_cost to zero to observe how parallel seq scan performs in high selectivity. Forgot to mention in the earlier mail. Overall the parallel seq scan performance is good. >> And also attached perf results for selectivity of 0.1 million and 5.4 >> million cases for analysis. >> > > I have checked perf reports and it seems that when selectivity is more, it > seems to be spending time in some kernel calls which could be due > communication of tuples. Yes. And also in low selectivity with increase of workers, tas and s_lock functions usage is getting increased. May be these are also one of the reasons for scaling problem. Regards, Hari Babu Fujitsu Australia
On Sat, Sep 19, 2015 at 1:45 AM, Robert Haas <robertmhaas@gmail.com> wrote: > On Fri, Sep 18, 2015 at 4:03 AM, Haribabu Kommi > <kommi.haribabu@gmail.com> wrote: >> On Thu, Sep 3, 2015 at 8:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >>> >>> Attached, find the rebased version of patch. >> >> Here are the performance test results: > > Thanks, this is really interesting. I'm very surprised by how much > kernel overhead this shows. I wonder where that's coming from. The > writes to and reads from the shm_mq shouldn't need to touch the kernel > at all except for page faults; that's why I chose this form of IPC. > It could be that the signals which are sent for flow control are > chewing up a lot of cycles, but if that's the problem, it's not very > clear from here. copy_user_generic_string doesn't sound like > something related to signals. And why all the kernel time in > _spin_lock? Maybe perf -g would help us tease out where this kernel > time is coming from. copy_user_generic_string system call is because of file read operations. In my test, I gave the shared_buffers as 12GB with the table size of 18GB. To reduce the user of copy_user_generic_string by loading all the pages into shared buffers with different combinations of 12GB and 20GB shared_buffers settings. The _spin_lock calls are from the signals that are generated by the workers. With the increase of tuple queue size, there is a change in kernel system calls usage. Here I attached the perf reports collected for your reference with -g option. > Some of this may be due to rapid context switching. Suppose the > master process is the bottleneck. Then each worker will fill up the > queue and go to sleep. When the master reads a tuple, the worker has > to wake up and write a tuple, and then it goes back to sleep. This > might be an indication that we need a bigger shm_mq size. I think > that would be experimenting with: if we double or quadruple or > increase by 10x the queue size, what happens to performance? I tried with 1, 2, 4, 8 and 10 multiply factor for the tuple queue size and collected the performance readings. Summary of the results are: - There is not much change in low selectivity cases with the increase of tuple queue size. - Till 1.5 million selectivity, the time taken to execute a query is 8 workers < 4 workers < 2 workers with any tuple queue size. - with tuple queue multiply factor 4 (i.e 4 * tuple queue size) for selectivity greater than 1.5 million 4 workers < 2 workers < 8 workers - with tuple queue multiply factor 8 or 10 for selectivity greater than 1.5 million 2 workers < 4 workers < 8 workers - From the above performance readings, increase of tuple queue size gets benefited with lesser number of workers compared to higher number of workers. - May be the tuple queue size can be calculated automatically based on the selectivity, average tuple width and number of workers. - when the buffers are loaded into shared_buffers using prewarm utility, there is not much scaling is visible with the increase of workers. Performance report is attached for your reference. Apart from the performance, I have the following observations. Workers are getting started irrespective of the system load. If user configures 16 workers, but because of a sudden increase in the system load, there are only 2 or 3 cpu's are only IDLE. In this case, if any parallel seq scan eligible query is executed, the backend may start 16 workers thus it can lead to overall increase of system usage and may decrease the performance of the other backend sessions? If the query have two parallel seq scan plan nodes and how the workers will be distributed across the two nodes? Currently parallel_seqscan_degree is used per plan node, even if we change that to per query, I think we need a worker distribution logic, instead of using all workers by a single plan node. Select with a limit clause is having a performance drawback with parallel seq scan in some scenarios, because of very less selectivity compared to seq scan, it should be better if we document it. Users can take necessary actions based on that for the queries with limit clause. Regards, Hari Babu Fujitsu Australia
Attachment
>
> On Thu, Sep 17, 2015 at 4:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > I haven't studied the planner logic in enough detail yet to have a
> > clear opinion on this. But what I do think is that this is a very
> > good reason why we should bite the bullet and add outfuncs/readfuncs
> > support for all Plan nodes. Otherwise, we're going to have to scan
> > subplans for nodes we're not expecting to see there, which seems
> > silly. We eventually want to allow all of those nodes in the worker
> > anyway.
> >
>
> makes sense to me. There are 39 plan nodes and it seems we have
> support for all of them in outfuncs and needs to add for most of them
> in readfuncs.
>
Attached patch (read_funcs_v1.patch) contains support for all the plan
Attachment
On Tue, Sep 22, 2015 at 3:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached patch (read_funcs_v1.patch) contains support for all the plan > and other nodes (like SubPlan which could be required for worker) except > CustomScan node. It looks like you need to update the top-of-file comment for outfuncs.c. Doesn't _readCommonPlan() leak? I think we should avoid that. _readCommonScan() and _readJoin() are worse: they leak multiple objects. It should be simple enough to avoid this: just have your helper function take a Plan * as argument and then use READ_TEMP_LOCALS() rather than READ_LOCALS(). Then the caller can use READ_LOCALS, call the helper to fill in all the Plan fields, and then read the other stuff itself. Instead of passing the Plan down by casting, how about passing &local_node->plan? And similarly for scans and joins. readAttrNumberCols uses sizeof(Oid) instead of sizeof(AttrNumber). I still don't understand why we need to handle PlanInvalItem. Actually, come to think of it, I'm not sure we need PlannedStmt either. Let's leave those out; they seem like trouble. I think it would be worth doing something like this: #define READ_ATTRNUMBER_ARRAY(fldname, len) \ token = pg_strtok(&length); \ local_node->fldname = readAttrNumberCols(len); And similarly for READ_OID_ARRAY, READ_BOOL_ARRAY, READ_INT_ARRAY. In general these routines are in the same order as plannodes.h, which is good. But _readNestLoopParam is out of place. Can we move it just after _readNestLoop? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> > On Thu, Sep 17, 2015 at 4:44 PM, Robert Haas <robertmhaas@gmail.com> wrote: > > > > > > I haven't studied the planner logic in enough detail yet to have a > > > clear opinion on this. But what I do think is that this is a very > > > good reason why we should bite the bullet and add outfuncs/readfuncs > > > support for all Plan nodes. Otherwise, we're going to have to scan > > > subplans for nodes we're not expecting to see there, which seems > > > silly. We eventually want to allow all of those nodes in the worker > > > anyway. > > > > > > > makes sense to me. There are 39 plan nodes and it seems we have > > support for all of them in outfuncs and needs to add for most of them > > in readfuncs. > > > > Attached patch (read_funcs_v1.patch) contains support for all the plan > and other nodes (like SubPlan which could be required for worker) except > CustomScan node. CustomScan contains TextOutCustomScan and doesn't > contain corresponding Read function pointer, we could add the support for > same, but I am not sure if CustomScan is required to be passed to worker > in near future, so I am leaving it for now. > Oh... I did exactly duplicated job a few days before. https://github.com/kaigai/sepgsql/blob/readfuncs/src/backend/nodes/readfuncs.c Regarding of CustomScan node, I'd like to run on worker process as soon as possible once it gets supported. I'm highly motivated. Andres raised a related topic a few weeks before: http://www.postgresql.org/message-id/20150825181933.GA19326@awork2.anarazel.de Here are two issues: * How to reproduce "methods" pointer on another process. Extension may not be loaded via shared_preload_libraries. -> One solution is to provide a pair of library and symbol name of the method table, instead of the pointer. I think itis a reasonable idea. * How to treat additional output of TextOutCustomScan. -> Here are two solutions. (1) Mark TextOutCustomScan as an obsolete callback, however, it still makes Andres concern becausewe need to form/deform private data for copyObject safe. (2) Add TextReadCustomScan (and NodeEqualCustomScan?) callbackto process private fields. > To verify the patch, I have done 2 things, first I have added elog to > the newly supported read funcs and then in planner, I have used > nodeToString and stringToNode on planned_stmt and then used the > newly generated planned_stmt for further execution. After making these > changes, I have ran make check-world and ensures that it covers all the > newly added nodes. > > Note, that as we don't populate funcid's in expressions during read, the > same has to be updated by traversing the tree and updating in different > expressions based on node type. Attached patch (read_funcs_test_v1) > contains the changes required for testing the patch. I am not very sure > about what do about some of the ForeignScan fields (fdw_private) in order > to update the funcid as the data in those expressions could be FDW specific. > This is anyway for test, so doesn't matter much, but the same will be > required to support read of ForeignScan node by worker. > Because of interface contract, it is role of FDW driver to put nodes which are safe to copyObject on fdw_exprs and fdw_private field. Unless FDW driver does not violate, fdw_exprs and fdw_private shall be reproduced on the worker side. (Of course, we cannot guarantee nobody has local pointer on private field...) Sorry, I cannot understand the sentence of funcid population. It seems to me funcid is displayed as-is, and _readFuncExpr() does nothing special...? Robert Haas said: > I think it would be worth doing something like this: > > #define READ_ATTRNUMBER_ARRAY(fldname, len) \ > token = pg_strtok(&length); \ > local_node->fldname = readAttrNumberCols(len); > > And similarly for READ_OID_ARRAY, READ_BOOL_ARRAY, READ_INT_ARRAY. > Like this? https://github.com/kaigai/sepgsql/blob/readfuncs/src/backend/nodes/readfuncs.c#L133 I think outfuncs.c also have similar macro to centralize the format of array. Actually, most of boolean array are displayed using booltostr(), however, only _outMergeJoin() uses "%d" format to display boolean as integer. It is a bit inconsistent manner. Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
>
> On Tue, Sep 22, 2015 at 3:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> readAttrNumberCols uses sizeof(Oid) instead of sizeof(AttrNumber).
>
> I still don't understand why we need to handle PlanInvalItem.
As such this is not required, just to maintain consistency as I have added
>
> Actually, come to think of it, I'm not sure we need PlannedStmt
> either.
On Tue, Sep 22, 2015 at 9:18 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > PlannedStmt is needed because we are passing the same from master > to worker for execution and the reason was that Executor interfaces > expect it. I thought we were passing the Plan and then the worker was constructed a PlannedStmt around it. If we're passing the PlannedStmt then I guess we need PlanInvalItem too, since there is a list of those hanging off of the PlannedStmt. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Tue, Sep 22, 2015 at 9:12 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > Oh... I did exactly duplicated job a few days before. > https://github.com/kaigai/sepgsql/blob/readfuncs/src/backend/nodes/readfuncs.c Please post the patch here, and clarify that it is under the PostgreSQL license. > Regarding of CustomScan node, I'd like to run on worker process as soon as > possible once it gets supported. I'm highly motivated. Great. > Andres raised a related topic a few weeks before: > http://www.postgresql.org/message-id/20150825181933.GA19326@awork2.anarazel.de > > Here are two issues: > > * How to reproduce "methods" pointer on another process. Extension may not be > loaded via shared_preload_libraries. The parallel mode stuff already has code to make sure that the same libraries that were loaded in the original backend get loaded in the new one. But that's not going to make the same pointer valid there. > -> One solution is to provide a pair of library and symbol name of the method > table, instead of the pointer. I think it is a reasonable idea. I agree. > * How to treat additional output of TextOutCustomScan. > -> Here are two solutions. (1) Mark TextOutCustomScan as an obsolete callback, > however, it still makes Andres concern because we need to form/deform private > data for copyObject safe. (2) Add TextReadCustomScan (and NodeEqualCustomScan?) > callback to process private fields. I don't see how making it obsolete solves anything. Any node that wants to run in a worker needs to have outfuncs and readfuncs support. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> On Tue, Sep 22, 2015 at 9:12 PM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > > Oh... I did exactly duplicated job a few days before. > > > https://github.com/kaigai/sepgsql/blob/readfuncs/src/backend/nodes/readfuncs > .c > > Please post the patch here, and clarify that it is under the PostgreSQL license. > Of course. I intend to submit. > > Regarding of CustomScan node, I'd like to run on worker process as soon as > > possible once it gets supported. I'm highly motivated. > > Great. > > > Andres raised a related topic a few weeks before: > > > http://www.postgresql.org/message-id/20150825181933.GA19326@awork2.anarazel. > de > > > > Here are two issues: > > > > * How to reproduce "methods" pointer on another process. Extension may not be > > loaded via shared_preload_libraries. > > The parallel mode stuff already has code to make sure that the same > libraries that were loaded in the original backend get loaded in the > new one. But that's not going to make the same pointer valid there. > > > -> One solution is to provide a pair of library and symbol name of the method > > table, instead of the pointer. I think it is a reasonable idea. > > I agree. > > > * How to treat additional output of TextOutCustomScan. > > -> Here are two solutions. (1) Mark TextOutCustomScan as an obsolete callback, > > however, it still makes Andres concern because we need to form/deform private > > data for copyObject safe. (2) Add TextReadCustomScan (and > NodeEqualCustomScan?) > > callback to process private fields. > > I don't see how making it obsolete solves anything. Any node that > wants to run in a worker needs to have outfuncs and readfuncs support. > In actually, I'm inclined to the (2) rather than (1). In case of (2), we shall need two new callbacks on _copyCustomScan and _readCustomScan. I'll try to make up. Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
>
> On Tue, Sep 22, 2015 at 3:21 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached patch (read_funcs_v1.patch) contains support for all the plan
> > and other nodes (like SubPlan which could be required for worker) except
> > CustomScan node.
>
> It looks like you need to update the top-of-file comment for outfuncs.c.
>
> Doesn't _readCommonPlan() leak?
> _readCommonScan() and _readJoin() are worse: they leak multiple
> objects. It should be simple enough to avoid this: just have your
> helper function take a Plan * as argument and then use
> READ_TEMP_LOCALS() rather than READ_LOCALS(). Then the caller can use
> READ_LOCALS, call the helper to fill in all the Plan fields, and then
> read the other stuff itself.
>
> Instead of passing the Plan down by casting, how about passing
> &local_node->plan? And similarly for scans and joins.
>
> readAttrNumberCols uses sizeof(Oid) instead of sizeof(AttrNumber).
>
> I still don't understand why we need to handle PlanInvalItem.
> Actually, come to think of it, I'm not sure we need PlannedStmt
> either. Let's leave those out; they seem like trouble.
>
> I think it would be worth doing something like this:
>
> #define READ_ATTRNUMBER_ARRAY(fldname, len) \
> token = pg_strtok(&length); \
> local_node->fldname = readAttrNumberCols(len);
>
> And similarly for READ_OID_ARRAY, READ_BOOL_ARRAY, READ_INT_ARRAY.
>
> In general these routines are in the same order as plannodes.h, which
> is good. But _readNestLoopParam is out of place. Can we move it just
> after _readNestLoop?
>
Attachment
>
> > > On Thu, Sep 17, 2015 at 4:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:
>
> > To verify the patch, I have done 2 things, first I have added elog to
> > the newly supported read funcs and then in planner, I have used
> > nodeToString and stringToNode on planned_stmt and then used the
> > newly generated planned_stmt for further execution. After making these
> > changes, I have ran make check-world and ensures that it covers all the
> > newly added nodes.
> >
> > Note, that as we don't populate funcid's in expressions during read, the
> > same has to be updated by traversing the tree and updating in different
> > expressions based on node type. Attached patch (read_funcs_test_v1)
> > contains the changes required for testing the patch. I am not very sure
> > about what do about some of the ForeignScan fields (fdw_private) in order
> > to update the funcid as the data in those expressions could be FDW specific.
> > This is anyway for test, so doesn't matter much, but the same will be
> > required to support read of ForeignScan node by worker.
> >
> Because of interface contract, it is role of FDW driver to put nodes which
> are safe to copyObject on fdw_exprs and fdw_private field. Unless FDW driver
> does not violate, fdw_exprs and fdw_private shall be reproduced on the worker
> side. (Of course, we cannot guarantee nobody has local pointer on private
> field...)
> Sorry, I cannot understand the sentence of funcid population. It seems to me
> funcid is displayed as-is, and _readFuncExpr() does nothing special...?
>
> Robert Haas said:
> > I think it would be worth doing something like this:
> >
> > #define READ_ATTRNUMBER_ARRAY(fldname, len) \
> > token = pg_strtok(&length); \
> > local_node->fldname = readAttrNumberCols(len);
> >
> > And similarly for READ_OID_ARRAY, READ_BOOL_ARRAY, READ_INT_ARRAY.
> >
> Like this?
> https://github.com/kaigai/sepgsql/blob/readfuncs/src/backend/nodes/readfuncs.c#L133
>
> I think outfuncs.c also have similar macro to centralize the format of array.
> Actually, most of boolean array are displayed using booltostr(), however, only
> _outMergeJoin() uses "%d" format to display boolean as integer.
> It is a bit inconsistent manner.
>
Yes, I have also noticed the same and thought of sending a patch which I
On Wed, Sep 23, 2015 at 3:22 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Instead of passing the Plan down by casting, how about passing >> &local_node->plan? And similarly for scans and joins. > Changed as per suggestion. The point of this change was to make it so that we wouldn't need the casts any more. You changed it so we didn't, but then didn't actually get rid of them. I did that, tweaked a comment, and committed this. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Tue, Sep 22, 2015 at 3:14 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > copy_user_generic_string system call is because of file read operations. > In my test, I gave the shared_buffers as 12GB with the table size of 18GB. OK, cool. So that's actually good: all that work would have to be done either way, and parallelism lets several CPUs work on it at once. > The _spin_lock calls are from the signals that are generated by the workers. > With the increase of tuple queue size, there is a change in kernel system > calls usage. And this part is not so good: that's additional work created by parallelism that wouldn't have to be done if we weren't in parallel mode. Of course, it's impossible to eliminate that, but we should try to reduce it. > - From the above performance readings, increase of tuple queue size > gets benefited with lesser > number of workers compared to higher number of workers. That makes sense to me, because there's a separate queue for each worker. If we have more workers, then the total amount of queue space available rises in proportion to the number of workers available. > Workers are getting started irrespective of the system load. If user > configures 16 workers, but > because of a sudden increase in the system load, there are only 2 or 3 > cpu's are only IDLE. > In this case, if any parallel seq scan eligible query is executed, the > backend may start 16 workers > thus it can lead to overall increase of system usage and may decrease > the performance of the > other backend sessions? Yep, that could happen. It's something we should work on, but the first version isn't going to try to be that smart. It's similar to the problem we already have with work_mem, and I want to work on it, but we need to get this working first. > If the query have two parallel seq scan plan nodes and how the workers > will be distributed across > the two nodes? Currently parallel_seqscan_degree is used per plan > node, even if we change that > to per query, I think we need a worker distribution logic, instead of > using all workers by a single > plan node. Yes, we need that, too. Again, at some point. > Select with a limit clause is having a performance drawback with > parallel seq scan in some scenarios, > because of very less selectivity compared to seq scan, it should be > better if we document it. Users > can take necessary actions based on that for the queries with limit clause. This is something I want to think further about in the near future. We don't have a great plan for shutting down workers when no further tuples are needed because, for example, an upper node has filled a limit. That makes using parallel query in contexts like Limit and InitPlan significantly more costly than you might expect. Perhaps we should avoid parallel plans altogether in those contexts, or maybe there is some other approach that can work. I haven't figured it out yet. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > [ new patches ] More review comments: ExecParallelEstimate() and ExecParallelInitializeDSM() should use planstate_tree_walker to iterate over the whole planstate tree. That's because the parallel node need not be at the top of the tree. By using the walker, you'll find plan state nodes that need work of the relevant type even if they are deeply buried. The logic should be something like: if (node == NULL) return false; switch (nodeTag(node)) { ... /* parallel aware node enumerated here */ } return planstate_tree_walker(node, ExecParallelEstimate, context); The function signature should be changed to bool ExecParallelEstimate(PlanState *planstate, parallel_estimate_ctx *context) where parallel_estimate_ctx is a structure containing ParallelContext *context and Size *psize. The comment about handling only a few node types should go away, because by using the planstate_tree_walker we can iterate over anything. It looks to me like there would be trouble if an initPlan or subPlan were kept below a Funnel, or as I guess we're going to call it, a Gather node. That's because a SubPlan doesn't actually have a pointer to the node tree for the sub-plan in it. It just has an index into PlannedStmt.subplans. But create_parallel_worker_plannedstmt sets the subplans list to NIL. So that's not gonna work. Now maybe there's no way for an initPlan or a subPlan to creep down under the funnel, but I don't immediately see what would prevent it. + * There should be atleast thousand pages to scan for each worker. "at least a thousand" +cost_patialseqscan(Path *path, PlannerInfo *root, patial->partial. I also don't see where you are checking that a partial seq scan has nothing parallel-restricted in its quals. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > [ new patches ] Still more review comments: + /* Allow space for terminating zero-byte */ + size = add_size(size, 1); This is pointless. The length is already stored separately, and if it weren't, this wouldn't be adequate anyway because a varlena can contain NUL bytes. It won't if it's text, but it could be bytea or numeric or whatever. RestoreBoundParams is broken, because it can do unaligned reads, which will core dump on some architectures (and merely be slow on others). If there are two or more parameters, and the first one is a varlena with a length that is not a multiple of MAXIMUM_ALIGNOF, the second SerializedParamExternData will be misaligned. Also, it's pretty lame that we send the useless pointer even for a pass-by-reference data type and then overwrite the bad pointer with a good one a few lines later. It would be better to design the serialization format so that we don't send the bogus pointer over the wire in the first place. It's also problematic in my view that there is so much duplicated code here. SerializedParamExternData and SerializedParamExecData are very similar and there are large swaths of very similar code to handle each case. Both structures contain Datum value, Size length, bool isnull, and Oid ptype, albeit not in the same order for some reason. The only difference is that SerializedParamExternData contains uint16 pflags and SerializedParamExecData contains int paramid. I think we need to refactor this code to get rid of all this duplication. I suggest that we decide to represent a datum here in a uniform fashion, perhaps like this: First, store a 4-byte header word. If this is -2, the value is NULL and no data follows. If it's -1, the value is pass-by-value and sizeof(Datum) bytes follow. If it's >0, the value is pass-by-reference and the value gives the number of following bytes that should be copied into a brand-new palloc'd chunk. Using a format like this, we can serialize and restore datums in various contexts - including bind and exec params - without having to rewrite the code each time. For example, for param extern data, you can dump an array of all the ptypes and paramids and then follow it with all of the params one after another. For param exec data, you can dump an array of all the ptypes and paramids and then follow it with the values one after another. The code that reads and writes the datums in both cases can be the same. If we need to send datums in other contexts, we can use the same code for it. The attached patch - which I even tested! - shows what I have in mind. It can save and restore the ParamListInfo (bind parameters). I haven't tried to adapt it to the exec parameters because I don't quite understand what you are doing there yet, but you can see that the datum-serialization logic is separated from the stuff that knows about the details of ParamListInfo, so datumSerialize() should be reusable for other purposes. This also doesn't have the other problems mentioned above. Thoughts? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
>
> On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
I have fixed most of the review comments raised in this mail
>
> + if (es->analyze && nodeTag(plan) == T_Funnel)
>
> Why not IsA()?
>
> + FinishParallelSetupAndAccumStats((FunnelState *)planstate);
>
> Shouldn't there be a space before planstate?
>
> + /* inform executor to collect buffer usage stats from parallel workers. */
> + estate->total_time = queryDesc->totaltime ? 1 : 0;
>
> Boy, the comment sure doesn't seem to match the code.
>
> + * Accumulate the stats by parallel workers before stopping the
> + * node.
>
> Suggest: "Accumulate stats from parallel workers before stopping node".
>
> + * If we are not able to send the tuple, then we assume that
> + * destination has closed and we won't be able to send any more
> + * tuples so we just end the loop.
>
> Suggest: "If we are not able to send the tuple, we assume the
> destination has closed and no more tuples can be sent. If that's the
> case, end the loop."
>
> +static void
> +EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
> + List *serialized_param_exec_vals,
> + int instOptions, Size *params_size,
> + Size *params_exec_size);
> +static void
> +StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
> + List *serialized_param_exec_vals,
> + int instOptions, Size params_size,
> + Size params_exec_size,
> + char **inst_options_space,
> + char **buffer_usage_space);
>
> Whitespace doesn't look like PostgreSQL style. Maybe run pgindent on
> the newly-added files?
>
> +/*
> + * This is required for parallel plan execution to fetch the information
> + * from dsm.
> + */
>
> This comment doesn't really say anything. Can we get a better one?
>
> + /*
> + * We expect each worker to populate the BufferUsage structure
> + * allocated by master backend and then master backend will aggregate
> + * all the usage along with it's own, so account it for each worker.
> + */
>
> This also needs improvement. Especially because...
>
> + /*
> + * We expect each worker to populate the instrumentation structure
> + * allocated by master backend and then master backend will aggregate
> + * all the information, so account it for each worker.
> + */
>
> ...it's almost identical to this one.
>
> + * Store bind parameter's list in dynamic shared memory. This is
> + * used for parameters in prepared query.
>
> s/bind parameter's list/bind parameters/. I think you could drop the
> second sentence, too.
>
> + /*
> + * Store PARAM_EXEC parameters list in dynamic shared memory. This is
> + * used for evaluation plan->initPlan params.
> + */
>
> So is the previous block for PARAM_EXTERN and this is PARAM_EXEC? If
> so, maybe that could be more clearly laid out.
>
> +GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
>
> Could this be a static function? Will it really be needed outside this file?
>
> And is there any use case for letting some of the arguments be NULL?
> Seems kind of an awkward API.
>
> +bool
> +ExecParallelBufferUsageAccum(Node *node)
> +{
> + if (node == NULL)
> + return false;
> +
> + switch (nodeTag(node))
> + {
> + case T_FunnelState:
> + {
> + FinishParallelSetupAndAccumStats((FunnelState*)node);
> + return true;
> + }
> + break;
> + default:
> + break;
> + }
> +
> + (void) planstate_tree_walker((Node*)((PlanState *)node)->lefttree, NULL,
> + ExecParallelBufferUsageAccum, 0);
> + (void) planstate_tree_walker((Node*)((PlanState *)node)->righttree, NULL,
> + ExecParallelBufferUsageAccum, 0);
> + return false;
> +}
>
> This seems wacky. I mean, isn't the point of planstate_tree_walker()
> that the callback itself doesn't have to handle recursion like this?
> And if not, then this wouldn't be adequate anyway, because some
> planstate nodes have children that are not in lefttree or righttree
> (cf. explain.c).
>
> + currentRelation = ExecOpenScanRelation(estate,
> + ((SeqScan *)
> node->ss.ps.plan)->scanrelid,
> + eflags);
>
> I can't see how this can possibly be remotely correct. The funnel
> node shouldn't be limited to scanning a baserel (cf. fdw_scan_tlist).
>
> +void ExecAccumulateInstInfo(FunnelState *node)
>
> Another place where pgindent would help. There are a bunch of others
> I noticed too, but I'm just mentioning a few here to make the point.
>
> + buffer_usage_worker = (BufferUsage *)(buffer_usage + (i *
> sizeof(BufferUsage)));
>
> Cast it to a BufferUsage * first. Then you can use &foo[i] to find
> the i'th element.
>
> + /*
> + * Re-initialize the parallel context and workers to perform
> + * rescan of relation. We want to gracefully shutdown all the
> + * workers so that they should be able to propagate any error
> + * or other information to master backend before dying.
> + */
> + FinishParallelSetupAndAccumStats(node);
>
> Somehow, this makes me feel like that function is badly named.
>
> +/*
> + * _readPlanInvalItem
> + */
> +static PlanInvalItem *
> +_readPlanInvalItem(void)
> +{
> + READ_LOCALS(PlanInvalItem);
> +
> + READ_INT_FIELD(cacheId);
> + READ_UINT_FIELD(hashValue);
> +
> + READ_DONE();
> +}
>
> I don't see why we should need to be able to copy PlanInvalItems. In
> fact, it seems like a bad idea.
>
> +#parallel_setup_cost = 0.0 # same scale as above
> +#define DEFAULT_PARALLEL_SETUP_COST 0.0
>
> This value is probably a bit on the low side.
>
> +int parallel_seqscan_degree = 0;
>
> I think we should have a GUC for the maximum degree of parallelism in
> a query generally, not the maximum degree of parallel sequential scan.
>
> + if (parallel_seqscan_degree >= MaxConnections)
> + {
> + write_stderr("%s: parallel_scan_degree must be less than
> max_connections\n", progname);
> + ExitPostmaster(1);
> + }
>
> I think this check is thoroughly unnecessary. It's comparing to the
> wrong thing anyway, because what actually matters is
> max_worker_processes, not max_connections. But in any case there is
> no need for the check. If somebody stupidly tries an unreasonable
> value for the maximum degree of parallelism, they won't get that many
> workers, but nothing will break. It's no worse than setting any other
> query planner costing parameter to an insane value.
>
> --- a/src/include/access/heapam.h
> +++ b/src/include/access/heapam.h
> @@ -126,6 +126,7 @@ extern void heap_rescan_set_params(HeapScanDesc
> scan, ScanKey key,
> extern void heap_endscan(HeapScanDesc scan);
> extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
>
> +
> extern bool heap_fetch(Relation relation, Snapshot snapshot,
>
> Stray whitespace change.
>
> Nodes where you just want to let the walk continue shouldn't need to
> be enumerated; dispatching like this should be the default case.
> + case T_Result:
> + fix_opfuncids((Node*) (((Result
> *)node)->resconstantqual));
> + break;
> Seems similarly wrong.
> + * cost_patialseqscan
> Typo. The actual function name has the same typo.
> + num_parallel_workers = parallel_seqscan_degree;
> + if (parallel_seqscan_degree <= estimated_parallel_workers)
> + num_parallel_workers = parallel_seqscan_degree;
> + else
> + num_parallel_workers = estimated_parallel_workers;
> Use Min?
> + {
> + instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
> + *inst_options = *instoptions;
> + if (inst_options)
> Same pointer variable check, it should be if (*inst_options) as per the
> estimate and store functions.
Fixed.
> + if (funnelstate->ss.ps.ps_ProjInfo)
> + slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
> + else
> + slot = funnelstate->ss.ss_ScanTupleSlot;
> Currently, there will not be a projinfo for funnel node. So always it uses
> the scan tuple slot. In case if it is different, we need to add the ExecProject
> call in ExecFunnel function. Currently it is not present, either we can document
> it or add the function call.
Added an appropriate comment as discussed upthread.
> + if (parallel_seqscan_degree >= MaxConnections)
> + {
> + write_stderr("%s: parallel_scan_degree must be less than
> max_connections\n", progname);
> + ExitPostmaster(1);
> + }
> The error condition works only during server start. User still can set
> parallel seqscan degree
> more than max connection at super user session level and etc.
removed this check.
> + if (!parallelstmt->inst_options)
> + (*receiver->rDestroy) (receiver);
> Why only when there is no instruementation only, the receiver needs to
> be destroyed?
2. Add an ID to each plan node and use that ID as the DSM key.
EXPLAIN ANALYZE should show both the planned and actual values.
Attachment
On Thu, Sep 24, 2015 at 2:31 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I have fixed most of the review comments raised in this mail > as well as other e-mails and rebased the patch on commit- > 020235a5. Even though I have fixed many of the things, but > still quite a few comments are yet to be handled. This patch > is mainly a rebased version to ease the review. We can continue > to have discussion on the left over things and I will address > those in consecutive patches. Thanks for the update. Here are some more review comments: 1. parallel_seqscan_degree is still not what we should have here. As previously mentioned, I think we should have a GUC for the maximum degree of parallelism in a query generally, not the maximum degree of parallel sequential scan. 2. fix_node_funcids() can go away because of commit 9f1255ac859364a86264a67729dbd1a36dd63ff2. 3. cost_patialseqscan is still misspelled. I pointed this out before, too. 4. Much more seriously than any of the above, create_parallelscan_paths() generates plans that are badly broken: rhaas=# explain select * from pgbench_accounts where filler < random()::text; QUERYPLAN -----------------------------------------------------------------------------------------Funnel on pgbench_accounts (cost=0.00..35357.73rows=3333333 width=97) Filter: ((filler)::text < (random())::text) Number of Workers: 10 -> PartialSeq Scan on pgbench_accounts (cost=0.00..35357.73 rows=3333333 width=97) Filter: ((filler)::text < (random())::text) (5 rows) This is wrong both because random() is parallel-restricted and thus can't be executed in a parallel worker, and also because enforcing a volatile qual twice is no good. rhaas=# explain select * from pgbench_accounts where aid % 10000 = 0; QUERY PLAN ---------------------------------------------------------------------------------------Funnel on pgbench_accounts (cost=0.00..28539.55rows=50000 width=97) Filter: ((aid % 10000) = 0) Number of Workers: 10 -> Partial Seq Scan on pgbench_accounts (cost=0.00..28539.55 rows=50000 width=97) Filter: ((aid % 10000) = 0) (5 rows) This will work, but it's a bad plan because we shouldn't need to re-apply the filter condition in the parallel leader after we've already checked it in the worker. rhaas=# explain select * from pgbench_accounts a where a.bid not in (select bid from pgbench_branches); QUERY PLAN -------------------------------------------------------------------------------------------Funnel on pgbench_accounts a (cost=2.25..26269.07 rows=5000000 width=97) Filter: (NOT (hashed SubPlan 1)) Number of Workers: 10 -> Partial Seq Scanon pgbench_accounts a (cost=2.25..26269.07 rows=5000000 width=97) Filter: (NOT (hashed SubPlan 1)) SubPlan 1 -> Seq Scan on pgbench_branches (cost=0.00..2.00 rows=100 width=4) SubPlan 1 -> Seq Scan on pgbench_branches (cost=0.00..2.00 rows=100width=4) (9 rows) This will not work, because the subplan isn't available inside the worker. Trying it without the EXPLAIN crashes the server. This is more or less the same issue as one of the known issues you already mentioned, but I mention it again here because I think this case is closely related to the previous two. Basically, you need to have some kind of logic for deciding which things need to go below the funnel and which on the funnel itself. The stuff that's safe should get pushed down, and the stuff that's not safe should get attached to the funnel. The unsafe stuff is whatever contains references to initplans or subplans, or anything that contains parallel-restricted functions ... and there might be some other stuff, too, but at least those things. Instead of preventing initplans or subplans from getting pushed down into the funnel, we could instead try to teach the system to push them down. However, that's very complicated; e.g. a subplan that references a CTE isn't safe to push down, and a subplan that references another subplan must be pushed down if that other subplan is pushed down, and an initplan that contains volatile functions can't be pushed down because each worker would execute it separately and they might not all get the same answer, and an initplan that references a temporary table can't be pushed down because it can't be referenced from a worker. All in all, it seems better not to go there right now. Now, when you fix this, you're quickly going to run into another problem, which is that as you have this today, the funnel node does not actually enforce its filter condition, so the EXPLAIN plan is a dastardly lie. And when you try to fix that, you're going to run into a third problem, which is that the stuff the funnel node needs in order to evaluate its filter condition might not be in the partial seq scan's target list. So you need to fix both of those problems, too. Even if you cheat and just don't generate a parallel path at all except when all quals can be pushed down, you're still going to have to fix these problems: it's not OK to print out a filter condition on the funnel as if you were going to enforce it, and then not actually enforce it there. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ new patches ]
>
>
> It looks to me like there would be trouble if an initPlan or subPlan
> were kept below a Funnel, or as I guess we're going to call it, a
> Gather node. That's because a SubPlan doesn't actually have a pointer
> to the node tree for the sub-plan in it. It just has an index into
> PlannedStmt.subplans. But create_parallel_worker_plannedstmt sets the
> subplans list to NIL. So that's not gonna work. Now maybe there's no
> way for an initPlan or a subPlan to creep down under the funnel, but I
> don't immediately see what would prevent it.
>
+ /*
+ * Evaluate the InitPlan and pass the PARAM_EXEC params, so that
+ * values can be shared with worker backend. This is different from
+ * the way InitPlans are evaluated (lazy evaluation) at other places
+ * as instead of sharing the InitPlan to all the workers and let them
+ * execute, we pass the values which can be directly used by worker
+ * backends.
+ */
+ serialized_param_exec = ExecAndFormSerializeParamExec(econtext,
+ node->ss.ps.plan->lefttree->allParam);
>
> On Thu, Sep 24, 2015 at 2:31 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I have fixed most of the review comments raised in this mail
> > as well as other e-mails and rebased the patch on commit-
> > 020235a5. Even though I have fixed many of the things, but
> > still quite a few comments are yet to be handled. This patch
> > is mainly a rebased version to ease the review. We can continue
> > to have discussion on the left over things and I will address
> > those in consecutive patches.
>
> Thanks for the update. Here are some more review comments:
>
> 1. parallel_seqscan_degree is still not what we should have here. As
> previously mentioned, I think we should have a GUC for the maximum
> degree of parallelism in a query generally, not the maximum degree of
> parallel sequential scan.
>
> 2. fix_node_funcids() can go away because of commit
> 9f1255ac859364a86264a67729dbd1a36dd63ff2.
>
> 3. cost_patialseqscan is still misspelled. I pointed this out before, too.
>
> 4. Much more seriously than any of the above,
> create_parallelscan_paths() generates plans that are badly broken:
>
> rhaas=# explain select * from pgbench_accounts where filler < random()::text;
> QUERY PLAN
> -----------------------------------------------------------------------------------------
> Funnel on pgbench_accounts (cost=0.00..35357.73 rows=3333333 width=97)
> Filter: ((filler)::text < (random())::text)
> Number of Workers: 10
> -> Partial Seq Scan on pgbench_accounts (cost=0.00..35357.73
> rows=3333333 width=97)
> Filter: ((filler)::text < (random())::text)
> (5 rows)
>
> This is wrong both because random() is parallel-restricted and thus
> can't be executed in a parallel worker, and also because enforcing a
> volatile qual twice is no good.
>
During create_plan() phase, separate out the quals that needs to be
Basically PartialSeqScan node can contain two different list of quals,
Now, I think the above can only be possible under the assumption that
Unless there is a good way to achieve the partial execution of quals,
Usage of restricted functions in target list -
One way could be if target list contains any restricted function, then parallel
> rhaas=# explain select * from pgbench_accounts where aid % 10000 = 0;
> QUERY PLAN
> ---------------------------------------------------------------------------------------
> Funnel on pgbench_accounts (cost=0.00..28539.55 rows=50000 width=97)
> Filter: ((aid % 10000) = 0)
> Number of Workers: 10
> -> Partial Seq Scan on pgbench_accounts (cost=0.00..28539.55
> rows=50000 width=97)
> Filter: ((aid % 10000) = 0)
> (5 rows)
>
> This will work, but it's a bad plan because we shouldn't need to
> re-apply the filter condition in the parallel leader after we've
> already checked it in the worker.
>
> rhaas=# explain select * from pgbench_accounts a where a.bid not in
> (select bid from pgbench_branches);
> QUERY PLAN
> -------------------------------------------------------------------------------------------
> Funnel on pgbench_accounts a (cost=2.25..26269.07 rows=5000000 width=97)
> Filter: (NOT (hashed SubPlan 1))
> Number of Workers: 10
> -> Partial Seq Scan on pgbench_accounts a (cost=2.25..26269.07
> rows=5000000 width=97)
> Filter: (NOT (hashed SubPlan 1))
> SubPlan 1
> -> Seq Scan on pgbench_branches (cost=0.00..2.00 rows=100 width=4)
> SubPlan 1
> -> Seq Scan on pgbench_branches (cost=0.00..2.00 rows=100 width=4)
> (9 rows)
>
> This will not work, because the subplan isn't available inside the
> worker. Trying it without the EXPLAIN crashes the server. This is
> more or less the same issue as one of the known issues you already
> mentioned, but I mention it again here because I think this case is
> closely related to the previous two.
>
> Basically, you need to have some kind of logic for deciding which
> things need to go below the funnel and which on the funnel itself.
> The stuff that's safe should get pushed down, and the stuff that's not
> safe should get attached to the funnel. The unsafe stuff is whatever
> contains references to initplans or subplans, or anything that
> contains parallel-restricted functions ... and there might be some
> other stuff, too, but at least those things.
>
> Instead of preventing initplans or subplans from getting pushed down
> into the funnel, we could instead try to teach the system to push them
> down. However, that's very complicated; e.g. a subplan that
> references a CTE isn't safe to push down, and a subplan that
> references another subplan must be pushed down if that other subplan
> is pushed down, and an initplan that contains volatile functions can't
> be pushed down because each worker would execute it separately and
> they might not all get the same answer, and an initplan that
> references a temporary table can't be pushed down because it can't be
> referenced from a worker. All in all, it seems better not to go there
> right now.
>
> Now, when you fix this, you're quickly going to run into another
> problem, which is that as you have this today, the funnel node does
> not actually enforce its filter condition, so the EXPLAIN plan is a
> dastardly lie. And when you try to fix that, you're going to run into
> a third problem, which is that the stuff the funnel node needs in
> order to evaluate its filter condition might not be in the partial seq
> scan's target list. So you need to fix both of those problems, too.
> Even if you cheat and just don't generate a parallel path at all
> except when all quals can be pushed down, you're still going to have
> to fix these problems: it's not OK to print out a filter condition on
> the funnel as if you were going to enforce it, and then not actually
> enforce it there.
>
>
> On Thu, Sep 3, 2015 at 6:21 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ new patches ]
>
> Still more review comments:
>
> + /* Allow space for terminating zero-byte */
> + size = add_size(size, 1);
>
> This is pointless. The length is already stored separately, and if it
> weren't, this wouldn't be adequate anyway because a varlena can
> contain NUL bytes. It won't if it's text, but it could be bytea or
> numeric or whatever.
>
> RestoreBoundParams is broken, because it can do unaligned reads, which
> will core dump on some architectures (and merely be slow on others).
> If there are two or more parameters, and the first one is a varlena
> with a length that is not a multiple of MAXIMUM_ALIGNOF, the second
> SerializedParamExternData will be misaligned.
>
> Also, it's pretty lame that we send the useless pointer even for a
> pass-by-reference data type and then overwrite the bad pointer with a
> good one a few lines later. It would be better to design the
> serialization format so that we don't send the bogus pointer over the
> wire in the first place.
>
> It's also problematic in my view that there is so much duplicated code
> here. SerializedParamExternData and SerializedParamExecData are very
> similar and there are large swaths of very similar code to handle each
> case. Both structures contain Datum value, Size length, bool isnull,
> and Oid ptype, albeit not in the same order for some reason. The only
> difference is that SerializedParamExternData contains uint16 pflags
> and SerializedParamExecData contains int paramid. I think we need to
> refactor this code to get rid of all this duplication.
> we decide to represent a datum here in a uniform fashion, perhaps like
> this:
>
> and no data follows. If it's -1, the value is pass-by-value and
> sizeof(Datum) bytes follow. If it's >0, the value is
> pass-by-reference and the value gives the number of following bytes
> that should be copied into a brand-new palloc'd chunk.
>
> Using a format like this, we can serialize and restore datums in
> various contexts - including bind and exec params - without having to
> rewrite the code each time. For example, for param extern data, you
> can dump an array of all the ptypes and paramids and then follow it
> with all of the params one after another. For param exec data, you
> can dump an array of all the ptypes and paramids and then follow it
> with the values one after another. The code that reads and writes the
> datums in both cases can be the same. If we need to send datums in
> other contexts, we can use the same code for it.
>
> The attached patch - which I even tested! - shows what I have in mind.
> It can save and restore the ParamListInfo (bind parameters). I
> haven't tried to adapt it to the exec parameters because I don't quite
> understand what you are doing there yet, but you can see that the
> datum-serialization logic is separated from the stuff that knows about
> the details of ParamListInfo, so datumSerialize() should be reusable
> for other purposes.
> mentioned above.
>
I have a question here which is why this format doesn't have a similar problem
On Fri, Sep 25, 2015 at 12:00 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I think initPlan will work with the existing patches as we are always > executing it in master and then sending the result to workers. Refer > below code in funnel patch: Sure, *if* that's what we're doing, then it will work. But if an initPlan actually attaches below a funnel, then it will break. Maybe that can't happen; I'm just sayin' ... -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Sep 25, 2015 at 12:55 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > In the latest patch (parallel_seqscan_partialseqscan_v18.patch) posted by > me yesterday, this was fixed. Am I missing something or by any chance > you are referring to wrong version of patch You're right, I'm wrong. Sorry. > Yes, the patch needs more work in terms of dealing with parallel-restricted > expressions/functions. One idea which I have explored previously is > push down only safe clauses to workers (via partialseqscan node) and > execute restricted clauses in master (via Funnel node). My analysis > is as follows: > > Usage of restricted functions in quals- > During create_plan() phase, separate out the quals that needs to be > executed at funnel node versus quals that needs to be executed on > partial seq scan node (do something similar to what is done in > create_indexscan_plan for index and non-index quals). > > Basically PartialSeqScan node can contain two different list of quals, > one for non-restrictive quals and other for restrictive quals and then > Funnel node can retrieve restrictive quals from partialseqscan node, > assuming partialseqscan node is its left child. > > Now, I think the above can only be possible under the assumption that > partialseqscan node is always a left child of funnel node, however that is > not true because a gating node (Result node) can be added between the > two and tomorrow there could be more cases when other nodes will be > added between the two, if we consider the case of aggregation, the > situation will be more complex as before partial aggregation, all the > quals should be executed. What's the situation where the gating Result node sneaks in there? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Sep 25, 2015 at 7:46 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I have a question here which is why this format doesn't have a similar > problem > as the current version, basically in current patch the second read of > SerializedParamExternData can be misaligned and for same reason in your > patch the second read of Oid could by misaligned? memcpy() can cope with unaligned data; structure member assignment can't. I've worked some of this code over fairly heavily today and I'm pretty happy with how my copy of execParallel.c looks now, but I've run into one issue where I wanted to check with you. There are three places where Instrumentation can be attached to a query: a ResultRelInfo's ri_TrigInstrument (which doesn't matter for us because we don't support parallel write queries, and triggers don't run on reads), a PlanState's instrument, and a QueryDesc's total time. Your patch makes provision to copy ONE Instrumentation structure per worker back to the parallel leader. I assumed this must be the QueryDesc's totaltime, but it looks like it's actually the PlanState of the top node passed to the worker. That's of course no good if we ever push more than one node down to the worker, which we may very well want to do in the initial version, and surely want to do eventually. We can't just deal with the top node and forget all the others. Is that really what's happening here, or am I confused? Assuming I'm not confused, I'm planning to see about fixing this... -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Sep 25, 2015 at 12:55 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>
> > Yes, the patch needs more work in terms of dealing with parallel-restricted
> > expressions/functions. One idea which I have explored previously is
> > push down only safe clauses to workers (via partialseqscan node) and
> > execute restricted clauses in master (via Funnel node). My analysis
> > is as follows:
> >
> > Usage of restricted functions in quals-
> > During create_plan() phase, separate out the quals that needs to be
> > executed at funnel node versus quals that needs to be executed on
> > partial seq scan node (do something similar to what is done in
> > create_indexscan_plan for index and non-index quals).
> >
> > Basically PartialSeqScan node can contain two different list of quals,
> > one for non-restrictive quals and other for restrictive quals and then
> > Funnel node can retrieve restrictive quals from partialseqscan node,
> > assuming partialseqscan node is its left child.
> >
> > Now, I think the above can only be possible under the assumption that
> > partialseqscan node is always a left child of funnel node, however that is
> > not true because a gating node (Result node) can be added between the
> > two and tomorrow there could be more cases when other nodes will be
> > added between the two, if we consider the case of aggregation, the
> > situation will be more complex as before partial aggregation, all the
> > quals should be executed.
>
> What's the situation where the gating Result node sneaks in there?
>
The plan node structure will be something like
>
> On Fri, Sep 25, 2015 at 7:46 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I have a question here which is why this format doesn't have a similar
> > problem
> > as the current version, basically in current patch the second read of
> > SerializedParamExternData can be misaligned and for same reason in your
> > patch the second read of Oid could by misaligned?
>
> memcpy() can cope with unaligned data; structure member assignment can't.
>
> I've worked some of this code over fairly heavily today and I'm pretty
> happy with how my copy of execParallel.c looks now, but I've run into
> one issue where I wanted to check with you. There are three places
> where Instrumentation can be attached to a query: a ResultRelInfo's
> ri_TrigInstrument (which doesn't matter for us because we don't
> support parallel write queries, and triggers don't run on reads), a
> PlanState's instrument, and a QueryDesc's total time.
>
> Your patch makes provision to copy ONE Instrumentation structure per
> worker back to the parallel leader. I assumed this must be the
> QueryDesc's totaltime, but it looks like it's actually the PlanState
> of the top node passed to the worker. That's of course no good if we
> ever push more than one node down to the worker, which we may very
> well want to do in the initial version, and surely want to do
> eventually. We can't just deal with the top node and forget all the
> others. Is that really what's happening here, or am I confused?
>
> Assuming I'm not confused, I'm planning to see about fixing this...
>
Can't we just traverse the queryDesc->planstate tree and fetch/add
>
> On Fri, Sep 25, 2015 at 12:00 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I think initPlan will work with the existing patches as we are always
> > executing it in master and then sending the result to workers. Refer
> > below code in funnel patch:
>
> Sure, *if* that's what we're doing, then it will work. But if an
> initPlan actually attaches below a funnel, then it will break.
>
> On Sat, Sep 26, 2015 at 6:07 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
>
> > Assuming I'm not confused, I'm planning to see about fixing this...
> >
>
> Can't we just traverse the queryDesc->planstate tree and fetch/add
> all the instrument information if there are multiple nodes?
>
I think the above suggestion made by me won't work, because we want
On Sat, Sep 26, 2015 at 3:08 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> memcpy() can cope with unaligned data; structure member assignment can't. > > So doesn't coping means, it anyways have to have to pay the performance > penality to make it equivalent to aligned address access. Apart from that, > today I had read about memcpy's behaviour incase of unaligned address, > it seems from some of the information on net that it could be unsafe > [1],[2]. I'm not concerned about the performance penalty for unaligned access in this case; I'm concerned about the fact that on some platforms it causes a segmentation fault. The links you've provided there are examples of cases where that wasn't true, and people reported that as a bug in memcpy. > Yes, you have figured out correctly, I was under impression that we > will have single node execution in worker for first version and then > will extend it later. No, I really want it to work with multiple nodes from the start, and I've pretty much got that working here now. > QueryDesc's totaltime is for instrumentation information for plugin's > like pg_stat_statements and we need only the total buffer usage > of each worker to make it work as the other information is already > collected in master backend, so I think that should work as I have > written. I don't think that's right at all. First, an extension can choose to look at any part of the Instrumentation, not just the buffer usage. Secondly, the buffer usage inside QueryDesc's totaltime isn't the same as the global pgBufferUsage. >> Assuming I'm not confused, I'm planning to see about fixing this... > > Can't we just traverse the queryDesc->planstate tree and fetch/add > all the instrument information if there are multiple nodes? Well you need to add each node's information in each worker to the corresponding node in the leader. You're not just adding them all up. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Sep 26, 2015 at 8:38 AM, Robert Haas <robertmhaas@gmail.com> wrote: >> QueryDesc's totaltime is for instrumentation information for plugin's >> like pg_stat_statements and we need only the total buffer usage >> of each worker to make it work as the other information is already >> collected in master backend, so I think that should work as I have >> written. > > I don't think that's right at all. First, an extension can choose to > look at any part of the Instrumentation, not just the buffer usage. > Secondly, the buffer usage inside QueryDesc's totaltime isn't the same > as the global pgBufferUsage. Oh... but I'm wrong. As long as our local pgBufferUsage gets update correctly to incorporate the data from the other workers, the InstrStopNode(queryDesc->totaltime) will suck in those statistics. And the only other things getting updated are nTuples (which shouldn't count anything that the workers did), firsttuple (similarly), and counter (where the behavior is a bit more arguable, but just counting the master's wall-clock time is at least defensible). So now I think you're right: this should be OK. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Sep 26, 2015 at 10:16 AM, Robert Haas <robertmhaas@gmail.com> wrote: > On Sat, Sep 26, 2015 at 8:38 AM, Robert Haas <robertmhaas@gmail.com> wrote: >>> QueryDesc's totaltime is for instrumentation information for plugin's >>> like pg_stat_statements and we need only the total buffer usage >>> of each worker to make it work as the other information is already >>> collected in master backend, so I think that should work as I have >>> written. >> >> I don't think that's right at all. First, an extension can choose to >> look at any part of the Instrumentation, not just the buffer usage. >> Secondly, the buffer usage inside QueryDesc's totaltime isn't the same >> as the global pgBufferUsage. > > Oh... but I'm wrong. As long as our local pgBufferUsage gets update > correctly to incorporate the data from the other workers, the > InstrStopNode(queryDesc->totaltime) will suck in those statistics. > And the only other things getting updated are nTuples (which shouldn't > count anything that the workers did), firsttuple (similarly), and > counter (where the behavior is a bit more arguable, but just counting > the master's wall-clock time is at least defensible). So now I think > you're right: this should be OK. OK, so here's a patch extracted from your parallel_seqscan_partialseqscan_v18.patch with a fairly substantial amount of rework by me: - I left out the Funnel node itself; this is just the infrastructure portion of the patch. I also left out the stop-the-executor early stuff and the serialization of PARAM_EXEC values. I want to have those things, but I think they need more thought and study first. - I reorganized the code a fair amount into a former that I thought was clearer, and certainly is closer to what I did previously in parallel.c. I found your version had lots of functions with lots of parameters, and I found that made the logic difficult to follow, at least for me. As part of that, I munged the interface a bit so that execParallel.c returns a structure with a bunch of pointers in it instead of separately returning each one as an out parameter. I think that's cleaner. If we need to add more stuff in the future, that way we don't break existing callers. - I reworked the interface with instrument.c and tried to preserve something of an abstraction boundary there. I also changed the way that stuff accumulated statistics to include more things; I couldn't see any reason to make it as narrow as you had it. - I did a bunch of cosmetic cleanup, including changing function names and rewriting comments. - I replaced your code for serializing and restoring a ParamListInfo with my version. - I fixed the code so that it can handle collecting instrumentation data from multiple nodes, bringing all the data back to the leader and associating it with the right plan node. This involved giving every plan node a unique ID, as discussed with Tom on another recent thread. After I did all that, I wrote some test code, which is also attached here, that adds a new GUC force_parallel_worker. If you set that GUC, when you run a query, it'll run the query in a parallel worker and feed the results back to the master. I've tested this and it seems to work, at least on the queries where you'd expect it to work. It's just test code, so it doesn't have error checking or make any attempt not to push down queries that will fail in parallel mode. But you can use it to see what happens. You can also run queries under EXPLAIN ANALYZE this way and, lo and behold, the worker stats show up attached to the correct plan nodes. I intend to commit this patch (but not the crappy test code, of course) pretty soon, and then I'm going to start working on the portion of the patch that actually adds the Funnel node, which I think you are working on renaming to Gather. I think that getting that part committed is likely to be pretty straightforward; it doesn't need to do a lot more than call this stuff and tell it to go do its thing. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
>
> I intend to commit this patch (but not the crappy test code, of
> course) pretty soon, and then I'm going to start working on the
> portion of the patch that actually adds the Funnel node, which I think
> you are working on renaming to Gather.
Attachment
On Tue, Sep 29, 2015 at 12:39 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached patch is a rebased patch based on latest commit (d1b7c1ff) > for Gather node. > > - I have to reorganize the defines in execParallel.h and .c. To keep > ParallelExecutorInfo, in GatherState node, we need to include execParallel.h > in execnodes.h which was creating compilation issues as execParallel.h > also includes execnodes.h, so for now I have defined ParallelExecutorInfo > in execnodes.h and instrumentation related structures in instrument.h. > - Renamed parallel_seqscan_degree to degree_of_parallelism > - Rename Funnel to Gather > - Removed PARAM_EXEC parameter handling code, I think we can do this > separately. > > I have to work more on partial seq scan patch for rebasing it and handling > review comments for the same, so for now I am sending the first part of > patch (which included Gather node functionality and some general support > for parallel-query execution). Thanks for the fast rebase. This patch needs a bunch of cleanup: - The formatting for the GatherState node's comment block is unlike that of surrounding comment blocks. It lacks the ------- dividers, and the indentation is not the same. Also, it refers to ParallelExecutorInfo by the type name, but the other members by structure member name. The convention is to refer to them by structure member name, so please do that. - The naming of fs_workersReady is inconsistent with the other structure members. The other members use all lower-case names, separating words with dashes, but this one uses a capital letter. The other members also don't prefix the names with anything, but this uses a "fs_" prefix which I assume is left over from when this was called FunnelState. Finally, this doesn't actually tell you when workers are ready, just whether they were launched. I suggest we rename this to "any_worker_launched". - Instead of moving the declaration of ParallelExecutorInfo, please just refer to it as "struct ParallelExecutorInfo" in execnodes.h. That way, you're not sucking these includes into all kinds of places they don't really need to be. - Let's not create a new PARALLEL_QUERY category of GUC. Instead, let's the GUC for the number of workers with under resource usage -> asynchronous behavior. - I don't believe that shm_toc *toc has any business being part of a generic PlanState node. At most, it should be part of an individual type of PlanState, like a GatherState or PartialSeqScanState. But really, I don't see why we need it there at all. It should, I think, only be needed during startup to dig out the information we need. So we should just dig that stuff out and keep pointers to whatever we actually need - in this case the ParallelExecutorInfo, I think - in the particular type of PlanState node that's at issue - here GatherState. After that we don't need a pointer to the toc any more. - I'd like to do some renaming of the new GUCs. I suggest we rename cpu_tuple_comm_cost to parallel_tuple_cost and degree_of_parallelism to max_parallel_degree. - I think that a Gather node should inherit from Plan, not Scan. A Gather node really shouldn't have a scanrelid. Now, admittedly, if the only thing under the Gather is a Partial Seq Scan, it wouldn't be totally bonkers to think of the Gather as scanning the same relation that the Partial Seq Scan is scanning. But in any more complex case, like where it's scanning a join, you're out of luck. You'll have to set scanrelid == 0, I suppose, but then, for example, ExecScanReScan is not going to work. In fact, as far as I can see, the only way nodeGather.c is actually using any of the generic scan stuff is by calling ExecInitScanTupleSlot, which is all of one line of code. ExecEndGather fetches node->ss.ss_currentRelation but then does nothing with it. So I think this is just a holdover from early version of this patch where what's now Gather and PartialSeqScan were a single node, and I think we should rip it out. - On a related note, the assertions in cost_gather() are both bogus and should be removed. Similarly with create_gather_plan(). As previously mentioned, the Gather node should not care what sort of thing is under it; I am not interested in restricting it to baserels and then undoing that later. - For the same reason, cost_gather() should refer to it's third argument as "rel" not "baserel". - Also, I think this stuff about physical tlists in create_gather_plan() is bogus. use_physical_tlist is ignorant of the possibility that the RelOptInfo passed to it might be anything other than a baserel, and I think it won't be happy if it gets a joinrel. Moreover, I think our plan here is that, at least for now, the Gather's tlist will always match the tlist of its child. If that's so, there's no point to this: it will end up with the same tlist either way. If any projection is needed, it should be done by the Gather node's child, not the Gather node itself. - Let's rename DestroyParallelSetupAndAccumStats to ExecShutdownGather. Instead of encasing the entire function in if statement, let's start with if (node->pei == NULL || node->pei->pcxt == NULL) return. - ExecParallelBufferUsageAccum should be declared to take an argument of type PlanState, not Node. Then you don't have to cast what you are passing to it, and it doesn't have to cast before calling itself. And, let's also rename it to ExecShutdownNode and move it to execProcnode.c. Having a "shutdown phase" that stops a node from asynchronously consuming additional resources could be useful for non-parallel node types - especially ForeignScan and CustomScan. And we could eventually extend this to be called in other situations, like when a Limit is filled give everything beneath it a chance to ease up. We don't have to do those bits of work right now but it seems well worth making this look like a generic facility. - Calling DestroyParallelSetupAndAccumStats from ExplainNode when we actually reach the Gather node is much too late. We should really be shutting down parallel workers at the end of the ExecutorRun phase, or certainly no later than ExecutorFinish. In fact, you have standard_ExecutorRun calling ExecParallelBufferUsageAccum() but only if queryDesc->totaltime is set. What I think you should do instead is call ExecShutdownNode a few lines earlier, before shutting down the tuple receiver, and do so unconditionally. That way, the workers are always shut down in the ExecutorRun phase, which should eliminate the need for this bit in explain.c. - The changes to postmaster.c and postgres.c consist of only additional #includes. Those can, presumably, be reverted. Other than that, hah hah, it looks pretty cool. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Sep 24, 2015 at 2:31 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > [ parallel_seqscan_partialseqscan_v18.patch ] I spent a bit of time reviewing the heapam.c changes in this patch this evening, and I think that your attempt to add support for synchronized scans has some problems. - In both heapgettup() and heapgettup_pagemode(), you call ss_report_location() on discovering that we're trying to initialize after the scan is already complete. This seems wrong. For the reasons noted in the existing comments, it's good for the backend that finishes the scan to report the starting position as the current position hint, but you don't want every parallel backend to do it turn. Unrelated, overlapping scans might be trying to continue advancing the scan, and you don't want to drag the position hint backward for no reason. - heap_parallelscan_initialize_startblock() calls ss_get_location() while holding a spinlock. This is clearly no good, because spinlocks can only be held while executing straight-line code that does not itself acquire locks - and ss_get_location() takes an *LWLock*. Among other problems, an error anywhere inside ss_get_location() would leave behind a stuck spinlock. - There's no point that I can see in initializing rs_startblock at all when a ParallelHeapScanDesc is in use. The ParallelHeapScanDesc, not rs_startblock, is going to tell you what page to scan next. I think heap_parallelscan_initialize_startblock() should basically do this, in the synchronized scan case: SpinLockAcquire(¶llel_scan->phs_mutex); page = parallel_scan->phs_startblock; SpinLockRelease(¶llel_scan->phs_mutex); if (page != InvalidBlockNumber) return; /* some other process already did this */ page = ss_get_location(scan->rs_rd, scan->rs_nblocks); SpinLockAcquire(¶llel_scan->phs_mutex); /* even though we checked before, someone might have beaten us here */ if (parallel_scan->phs_startblock == InvalidBlockNumber) { parallel_scan->phs_startblock = page; parallel_scan->phs_cblock = page; } SpinLockRelease(¶llel_scan->phs_mutex); - heap_parallelscan_nextpage() seems to have gotten unnecessarily complicated. I particularly dislike the way you increment phs_cblock and then sometimes try to back it out later. Let's decide that phs_cblock == InvalidBlockNumber means that the scan is finished, while phs_cblock == phs_startblock means that we're just starting. We then don't need phs_firstpass at all, and can write: SpinLockAcquire(¶llel_scan->phs_mutex); page = parallel_scan->phs_cblock; if (page != InvalidBlockNumber) { parallel_scan->phs_cblock++; if (parallel_scan->phs_cblock >= scan->rs_nblocks) parallel_scan->phs_cblock = 0; if (parallel_scan->phs_cblock == parallel_scan->phs_startblock) { parallel_scan->phs_cblock = InvalidBlockNumber; report_scan_done = true; } } SpinLockRelease(¶llel_scan->phs_mutex); At this point, if page contains InvalidBlockNumber, then the scan is done, and if it contains anything else, that's the next page that the current process should scan. If report_scan_done is true, we are the first to observe that the scan is done and should call ss_report_location(). -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Tue, Sep 29, 2015 at 12:39 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached patch is a rebased patch based on latest commit (d1b7c1ff)
> > for Gather node.
> >
> > - I have to reorganize the defines in execParallel.h and .c. To keep
> > ParallelExecutorInfo, in GatherState node, we need to include execParallel.h
> > in execnodes.h which was creating compilation issues as execParallel.h
> > also includes execnodes.h, so for now I have defined ParallelExecutorInfo
> > in execnodes.h and instrumentation related structures in instrument.h.
> > - Renamed parallel_seqscan_degree to degree_of_parallelism
> > - Rename Funnel to Gather
> > - Removed PARAM_EXEC parameter handling code, I think we can do this
> > separately.
> >
> > I have to work more on partial seq scan patch for rebasing it and handling
> > review comments for the same, so for now I am sending the first part of
> > patch (which included Gather node functionality and some general support
> > for parallel-query execution).
>
> Thanks for the fast rebase.
>
> This patch needs a bunch of cleanup:
>
> - The formatting for the GatherState node's comment block is unlike
> that of surrounding comment blocks. It lacks the ------- dividers,
> and the indentation is not the same. Also, it refers to
> ParallelExecutorInfo by the type name, but the other members by
> structure member name. The convention is to refer to them by
> structure member name, so please do that.
>
> - The naming of fs_workersReady is inconsistent with the other
> structure members. The other members use all lower-case names,
> separating words with dashes, but this one uses a capital letter. The
> other members also don't prefix the names with anything, but this uses
> a "fs_" prefix which I assume is left over from when this was called
> FunnelState. Finally, this doesn't actually tell you when workers are
> ready, just whether they were launched. I suggest we rename this to
> "any_worker_launched".
>
> - Instead of moving the declaration of ParallelExecutorInfo, please
> just refer to it as "struct ParallelExecutorInfo" in execnodes.h.
> That way, you're not sucking these includes into all kinds of places
> they don't really need to be.
>
> - Let's not create a new PARALLEL_QUERY category of GUC. Instead,
> let's the GUC for the number of workers with under resource usage ->
> asynchronous behavior.
>
> - I don't believe that shm_toc *toc has any business being part of a
> generic PlanState node. At most, it should be part of an individual
> type of PlanState, like a GatherState or PartialSeqScanState. But
> really, I don't see why we need it there at all.
> - I'd like to do some renaming of the new GUCs. I suggest we rename
> cpu_tuple_comm_cost to parallel_tuple_cost and degree_of_parallelism
> to max_parallel_degree.
>
> - I think that a Gather node should inherit from Plan, not Scan. A
> Gather node really shouldn't have a scanrelid. Now, admittedly, if
> the only thing under the Gather is a Partial Seq Scan, it wouldn't be
> totally bonkers to think of the Gather as scanning the same relation
> that the Partial Seq Scan is scanning. But in any more complex case,
> like where it's scanning a join, you're out of luck. You'll have to
> set scanrelid == 0, I suppose, but then, for example, ExecScanReScan
> is not going to work. In fact, as far as I can see, the only way
> nodeGather.c is actually using any of the generic scan stuff is by
> calling ExecInitScanTupleSlot, which is all of one line of code.
> ExecEndGather fetches node->ss.ss_currentRelation but then does
> nothing with it. So I think this is just a holdover from early
> version of this patch where what's now Gather and PartialSeqScan were
> a single node, and I think we should rip it out.
>
> - On a related note, the assertions in cost_gather() are both bogus
> and should be removed. Similarly with create_gather_plan(). As
> previously mentioned, the Gather node should not care what sort of
> thing is under it; I am not interested in restricting it to baserels
> and then undoing that later.
>
> - For the same reason, cost_gather() should refer to it's third
> argument as "rel" not "baserel".
>
> - Also, I think this stuff about physical tlists in
> create_gather_plan() is bogus. use_physical_tlist is ignorant of the
> possibility that the RelOptInfo passed to it might be anything other
> than a baserel, and I think it won't be happy if it gets a joinrel.
> Moreover, I think our plan here is that, at least for now, the
> Gather's tlist will always match the tlist of its child. If that's
> so, there's no point to this: it will end up with the same tlist
> either way. If any projection is needed, it should be done by the
> Gather node's child, not the Gather node itself.
>
> - Let's rename DestroyParallelSetupAndAccumStats to
> ExecShutdownGather. Instead of encasing the entire function in if
> statement, let's start with if (node->pei == NULL || node->pei->pcxt
> == NULL) return.
>
> - ExecParallelBufferUsageAccum should be declared to take an argument
> of type PlanState, not Node. Then you don't have to cast what you are
> passing to it, and it doesn't have to cast before calling itself. And,
> let's also rename it to ExecShutdownNode and move it to
> execProcnode.c. Having a "shutdown phase" that stops a node from
> asynchronously consuming additional resources could be useful for
> non-parallel node types - especially ForeignScan and CustomScan. And
> we could eventually extend this to be called in other situations, like
> when a Limit is filled give everything beneath it a chance to ease up.
> We don't have to do those bits of work right now but it seems well
> worth making this look like a generic facility.
>
> - Calling DestroyParallelSetupAndAccumStats from ExplainNode when we
> actually reach the Gather node is much too late. We should really be
> shutting down parallel workers at the end of the ExecutorRun phase, or
> certainly no later than ExecutorFinish. In fact, you have
> standard_ExecutorRun calling ExecParallelBufferUsageAccum() but only
> if queryDesc->totaltime is set. What I think you should do instead is
> call ExecShutdownNode a few lines earlier, before shutting down the
> tuple receiver, and do so unconditionally. That way, the workers are
> always shut down in the ExecutorRun phase, which should eliminate the
> need for this bit in explain.c.
>
> - The changes to postmaster.c and postgres.c consist of only
> additional #includes. Those can, presumably, be reverted.
>
Attachment
On Wed, Sep 30, 2015 at 11:23 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> - I don't believe that shm_toc *toc has any business being part of a >> generic PlanState node. At most, it should be part of an individual >> type of PlanState, like a GatherState or PartialSeqScanState. But >> really, I don't see why we need it there at all. > > We need it for getting parallelheapscan descriptor in case of > partial sequence scan node, it doesn't seem like a good idea > to retrieve it in begining, as we need to dig into plan tree to > get the node_id for getting the value of parallelheapscan descriptor > from toc. > > Now, I think we can surely keep it in PartialSeqScanState or any > other node state which might need it later, but I felt this is quite > generic and we might need to fetch node specific information from toc > going forward. It's true that the PartialSeqScanState will need a way to get at the toc, but I don't think that means we should stash it in the PlanState. I've taken that part out for now. >> - I think that a Gather node should inherit from Plan, not Scan. A >> Gather node really shouldn't have a scanrelid. Now, admittedly, if >> the only thing under the Gather is a Partial Seq Scan, it wouldn't be >> totally bonkers to think of the Gather as scanning the same relation >> that the Partial Seq Scan is scanning. But in any more complex case, >> like where it's scanning a join, you're out of luck. You'll have to >> set scanrelid == 0, I suppose, but then, for example, ExecScanReScan >> is not going to work. In fact, as far as I can see, the only way >> nodeGather.c is actually using any of the generic scan stuff is by >> calling ExecInitScanTupleSlot, which is all of one line of code. >> ExecEndGather fetches node->ss.ss_currentRelation but then does >> nothing with it. So I think this is just a holdover from early >> version of this patch where what's now Gather and PartialSeqScan were >> a single node, and I think we should rip it out. > > makes sense and I think GatherState should also be inherit from PlanState > instead of ScanState which I have changed in patch attached. You missed a number of things while doing this - I cleaned them up. >> - Also, I think this stuff about physical tlists in >> create_gather_plan() is bogus. use_physical_tlist is ignorant of the >> possibility that the RelOptInfo passed to it might be anything other >> than a baserel, and I think it won't be happy if it gets a joinrel. >> Moreover, I think our plan here is that, at least for now, the >> Gather's tlist will always match the tlist of its child. If that's >> so, there's no point to this: it will end up with the same tlist >> either way. If any projection is needed, it should be done by the >> Gather node's child, not the Gather node itself. > > Yes, Gather node itself doesn't need to do projection, but it > needs the projection info to store the same in Slot after fetching > the tuple from tuple queue. Now this is not required for Gather > node itself, but it might be required for any node on top of > Gather node. > > Here, I think one thing we could do is that use the subplan's target > list as currently is being done for quals. The only risk is what if > Gating node is added on top of partialseqscan (subplan), but I have checked > that is safe, because Gating plan uses the same target list as it's child. > Also I don't think we need to process any quals at Gather node, so I will > make that as Null, I will do this change in next version unless you see > any problem with it. > > Yet another idea is during set_plan_refs(), we can assign leftchild's > target list to parent in case of Gather node (right now it's done in > reverse way which needs to be changed.) > > What is your preference? I made it work like other nodes that inherit their left child's target list. I made a few other changes as well: - I wrote documentation for the GUCs. This probably needs to be expanded once we get the whole feature in, but it's something. - I added a new single_copy option to the gather. A single-copy gather never tries to execute the plan itself, unless it can't get any workers. This is very handy for testing, since it lets you stick a Gather node on top of an arbitrary plan and, if everything's working, it should work just as if the Gather node weren't there. I did a bit of minor fiddling with the contents of the GatherState to make this work. It's also useful in real life, since somebody can stick a single-copy Gather node into a plan someplace and run everything below that in a worker. - I fixed a bug in ExecGather - you were testing whether node->pei->pcxt is NULL, which seg faults on the first time through. The correct thing is to node->pei. - Assorted cosmetic changes. - I again left out the early-executor-stop stuff, preferring to leave that for a separate commit. That done, I have committed this. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Hi Robert, Gather node was oversight by readfunc.c, even though it shall not be serialized actually. Also, it used incompatible WRITE_xxx_FIELD() macro on outfuncs.c. The attached patch fixes both of incomsistence. Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com> > -----Original Message----- > From: pgsql-hackers-owner@postgresql.org > [mailto:pgsql-hackers-owner@postgresql.org] On Behalf Of Robert Haas > Sent: Wednesday, September 30, 2015 2:19 AM > To: Amit Kapila > Cc: Kaigai Kouhei(海外 浩平); Haribabu Kommi; Gavin Flower; Jeff Davis; Andres > Freund; Amit Langote; Amit Langote; Fabrízio Mello; Thom Brown; Stephen Frost; > pgsql-hackers > Subject: Re: [HACKERS] Parallel Seq Scan > > On Tue, Sep 29, 2015 at 12:39 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > > Attached patch is a rebased patch based on latest commit (d1b7c1ff) > > for Gather node. > > > > - I have to reorganize the defines in execParallel.h and .c. To keep > > ParallelExecutorInfo, in GatherState node, we need to include execParallel.h > > in execnodes.h which was creating compilation issues as execParallel.h > > also includes execnodes.h, so for now I have defined ParallelExecutorInfo > > in execnodes.h and instrumentation related structures in instrument.h. > > - Renamed parallel_seqscan_degree to degree_of_parallelism > > - Rename Funnel to Gather > > - Removed PARAM_EXEC parameter handling code, I think we can do this > > separately. > > > > I have to work more on partial seq scan patch for rebasing it and handling > > review comments for the same, so for now I am sending the first part of > > patch (which included Gather node functionality and some general support > > for parallel-query execution). > > Thanks for the fast rebase. > > This patch needs a bunch of cleanup: > > - The formatting for the GatherState node's comment block is unlike > that of surrounding comment blocks. It lacks the ------- dividers, > and the indentation is not the same. Also, it refers to > ParallelExecutorInfo by the type name, but the other members by > structure member name. The convention is to refer to them by > structure member name, so please do that. > > - The naming of fs_workersReady is inconsistent with the other > structure members. The other members use all lower-case names, > separating words with dashes, but this one uses a capital letter. The > other members also don't prefix the names with anything, but this uses > a "fs_" prefix which I assume is left over from when this was called > FunnelState. Finally, this doesn't actually tell you when workers are > ready, just whether they were launched. I suggest we rename this to > "any_worker_launched". > > - Instead of moving the declaration of ParallelExecutorInfo, please > just refer to it as "struct ParallelExecutorInfo" in execnodes.h. > That way, you're not sucking these includes into all kinds of places > they don't really need to be. > > - Let's not create a new PARALLEL_QUERY category of GUC. Instead, > let's the GUC for the number of workers with under resource usage -> > asynchronous behavior. > > - I don't believe that shm_toc *toc has any business being part of a > generic PlanState node. At most, it should be part of an individual > type of PlanState, like a GatherState or PartialSeqScanState. But > really, I don't see why we need it there at all. It should, I think, > only be needed during startup to dig out the information we need. So > we should just dig that stuff out and keep pointers to whatever we > actually need - in this case the ParallelExecutorInfo, I think - in > the particular type of PlanState node that's at issue - here > GatherState. After that we don't need a pointer to the toc any more. > > - I'd like to do some renaming of the new GUCs. I suggest we rename > cpu_tuple_comm_cost to parallel_tuple_cost and degree_of_parallelism > to max_parallel_degree. > > - I think that a Gather node should inherit from Plan, not Scan. A > Gather node really shouldn't have a scanrelid. Now, admittedly, if > the only thing under the Gather is a Partial Seq Scan, it wouldn't be > totally bonkers to think of the Gather as scanning the same relation > that the Partial Seq Scan is scanning. But in any more complex case, > like where it's scanning a join, you're out of luck. You'll have to > set scanrelid == 0, I suppose, but then, for example, ExecScanReScan > is not going to work. In fact, as far as I can see, the only way > nodeGather.c is actually using any of the generic scan stuff is by > calling ExecInitScanTupleSlot, which is all of one line of code. > ExecEndGather fetches node->ss.ss_currentRelation but then does > nothing with it. So I think this is just a holdover from early > version of this patch where what's now Gather and PartialSeqScan were > a single node, and I think we should rip it out. > > - On a related note, the assertions in cost_gather() are both bogus > and should be removed. Similarly with create_gather_plan(). As > previously mentioned, the Gather node should not care what sort of > thing is under it; I am not interested in restricting it to baserels > and then undoing that later. > > - For the same reason, cost_gather() should refer to it's third > argument as "rel" not "baserel". > > - Also, I think this stuff about physical tlists in > create_gather_plan() is bogus. use_physical_tlist is ignorant of the > possibility that the RelOptInfo passed to it might be anything other > than a baserel, and I think it won't be happy if it gets a joinrel. > Moreover, I think our plan here is that, at least for now, the > Gather's tlist will always match the tlist of its child. If that's > so, there's no point to this: it will end up with the same tlist > either way. If any projection is needed, it should be done by the > Gather node's child, not the Gather node itself. > > - Let's rename DestroyParallelSetupAndAccumStats to > ExecShutdownGather. Instead of encasing the entire function in if > statement, let's start with if (node->pei == NULL || node->pei->pcxt > == NULL) return. > > - ExecParallelBufferUsageAccum should be declared to take an argument > of type PlanState, not Node. Then you don't have to cast what you are > passing to it, and it doesn't have to cast before calling itself. And, > let's also rename it to ExecShutdownNode and move it to > execProcnode.c. Having a "shutdown phase" that stops a node from > asynchronously consuming additional resources could be useful for > non-parallel node types - especially ForeignScan and CustomScan. And > we could eventually extend this to be called in other situations, like > when a Limit is filled give everything beneath it a chance to ease up. > We don't have to do those bits of work right now but it seems well > worth making this look like a generic facility. > > - Calling DestroyParallelSetupAndAccumStats from ExplainNode when we > actually reach the Gather node is much too late. We should really be > shutting down parallel workers at the end of the ExecutorRun phase, or > certainly no later than ExecutorFinish. In fact, you have > standard_ExecutorRun calling ExecParallelBufferUsageAccum() but only > if queryDesc->totaltime is set. What I think you should do instead is > call ExecShutdownNode a few lines earlier, before shutting down the > tuple receiver, and do so unconditionally. That way, the workers are > always shut down in the ExecutorRun phase, which should eliminate the > need for this bit in explain.c. > > - The changes to postmaster.c and postgres.c consist of only > additional #includes. Those can, presumably, be reverted. > > Other than that, hah hah, it looks pretty cool. > > -- > 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
Attachment
>
> On Thu, Sep 24, 2015 at 2:31 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ parallel_seqscan_partialseqscan_v18.patch ]
>
> I spent a bit of time reviewing the heapam.c changes in this patch
> this evening, and I think that your attempt to add support for
> synchronized scans has some problems.
>
Thanks for the review and I agree with all the suggestions provided
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachment
On Thu, Oct 1, 2015 at 2:35 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > Gather node was oversight by readfunc.c, even though it shall not be > serialized actually. > Also, it used incompatible WRITE_xxx_FIELD() macro on outfuncs.c. > > The attached patch fixes both of incomsistence. Thanks. You missed READ_DONE() but fortunately my compiler noticed that oversight. Committed with that fix. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Oct 1, 2015 at 7:52 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Sep 30, 2015 at 7:05 AM, Robert Haas <robertmhaas@gmail.com> wrote: >> On Thu, Sep 24, 2015 at 2:31 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > [ parallel_seqscan_partialseqscan_v18.patch ] >> >> I spent a bit of time reviewing the heapam.c changes in this patch >> this evening, and I think that your attempt to add support for >> synchronized scans has some problems. > > Thanks for the review and I agree with all the suggestions provided > by you. Fixed all of them in attached patch > (parallel_seqscan_heapscan_v1.patch). Thanks. Does heap_parallelscan_nextpage really need a pscan_finished output parameter, or can it just return InvalidBlockNumber to indicate end of scan? I think the latter can be done and would be cleaner. There doesn't seem to be anything that ensures that everybody who's scanning the relation agrees on whether we're doing a synchronized scan. I think that heap_parallelscan_initialize() should taken an additional Boolean argument, allow_sync. It should decide whether to actually perform a syncscan using the logic from initscan(), and then it should store a phs_syncscan flag into the ParallelHeapScanDesc. heap_beginscan_internal should set rs_syncscan based on phs_syncscan, regardless of anything else. I think heap_parallel_rescan() is an unworkable API. When rescanning a synchronized scan, the existing logic keeps the original start-block so that the scan's results are reproducible, but no longer reports the scan position since we're presumably out of step with other backends. This isn't going to work at all with this API. I don't think you can swap out the ParallelHeapScanDesc for another one once you've installed it; the point of a rescan is that you are letting the HeapScanDesc (or ParallelHeapScanDesc) hold onto some state from the first time, and that doesn't work at all here. So, I think this function should just go away, and callers should be able to just use heap_rescan(). Now this presents a bit of a problem for PartialSeqScan, because, on a rescan, nodeGather.c completely destroys the DSM and creates a new one. I think we're going to need to change that. I think we can adapt the parallel context machinery so that after WaitForParallelWorkersToFinish(), you can again call LaunchParallelWorkers(). (That might already work, but I wouldn't be surprised if it doesn't quite work.) This would make rescans somewhat more efficient because we wouldn't have to destroy and re-create the DSM each time. It means that the DSM would have to stick around until we're totally done with the query, rather than going away when ExecGather() returns the last tuple, but that doesn't sound too bad. We can still clean up the workers when we've returned all the tuples, which I think is the most important thing. This is obviously going to present some design complications for the as-yet-uncommitted code to push down PARAM_EXEC parameters, because if the new value takes more bytes to store than the old value, there won't be room to update the existing DSM in place. There are several possible solutions to that problem, but the one that appeals to me most right now is just don't generate plans that would require that feature. It doesn't seem particularly appealing to me to put a Gather node on the inner side of a NestLoop -- at least not until we can execute that without restarting workers, which we're certainly some distance from today. And maybe not even then. For initPlans, the existing code might be adequate, because I think we never re-evaluate those. And for subPlans, we can potentially handle cases where each worker can evaluate the subPlan separately below the Gather; we just can't handle cases where the subPlan attaches above the Gather and is used below it. Or, we can get around these limitations by redesigning the PARAM_EXEC pushdown mechanism in some way. But even if we don't, it's not crippling. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> On Thu, Oct 1, 2015 at 2:35 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: > > Gather node was oversight by readfunc.c, even though it shall not be > > serialized actually. > > Also, it used incompatible WRITE_xxx_FIELD() macro on outfuncs.c. > > > > The attached patch fixes both of incomsistence. > > Thanks. You missed READ_DONE() but fortunately my compiler noticed > that oversight. Committed with that fix. > I could find one other strangenes, at explain.c. case T_Gather: { Gather *gather = (Gather *) plan; show_scan_qual(plan->qual, "Filter", planstate, ancestors, es); if (plan->qual) show_instrumentation_count("Rows Removed by Filter", 1, planstate, es); ExplainPropertyInteger("Number of Workers", gather->num_workers, es); if (gather->single_copy) ExplainPropertyText("Single Copy", gather->single_copy ? "true" : "false", es); } break; What is the intention of the last if-check? The single_copy is checked in the argument of ExplainPropertyText(). Thanks, -- NEC Business Creation Division / PG-Strom Project KaiGai Kohei <kaigai@ak.jp.nec.com>
On Fri, Oct 2, 2015 at 4:27 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: >> On Thu, Oct 1, 2015 at 2:35 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote: >> > Gather node was oversight by readfunc.c, even though it shall not be >> > serialized actually. >> > Also, it used incompatible WRITE_xxx_FIELD() macro on outfuncs.c. >> > >> > The attached patch fixes both of incomsistence. >> >> Thanks. You missed READ_DONE() but fortunately my compiler noticed >> that oversight. Committed with that fix. >> > I could find one other strangenes, at explain.c. > > case T_Gather: > { > Gather *gather = (Gather *) plan; > > show_scan_qual(plan->qual, "Filter", planstate, ancestors, es); > if (plan->qual) > show_instrumentation_count("Rows Removed by Filter", 1, > planstate, es); > ExplainPropertyInteger("Number of Workers", > gather->num_workers, es); > if (gather->single_copy) > ExplainPropertyText("Single Copy", > gather->single_copy ? "true" : "false", > es); > } > break; > > What is the intention of the last if-check? > The single_copy is checked in the argument of ExplainPropertyText(). Oops, that was dumb. single_copy only makes sense if num_workers == 1, so I intended the if-test to be based on num_workers, not single_copy. Not sure if we should just make that change now or if there's a better way to do display it. I'm sort of tempted to try to come up with a shorthand that only uses one line in text mode - e.g. set pname to something like "Gather 3" if there are 3 workers, "Gather 1" if there is worker, or "Gather Single" if there is one worker and we're in single_copy mode. In non-text mode, of course, the properties should be displayed separately, for machine parse-ability. But maybe I'm getting ahead of myself and we should just change it to if (gather->num_workers == 1) for now. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
>
> Does heap_parallelscan_nextpage really need a pscan_finished output
> parameter, or can it just return InvalidBlockNumber to indicate end of
> scan? I think the latter can be done and would be cleaner.
>
> scanning the relation agrees on whether we're doing a synchronized
> scan.
> additional Boolean argument, allow_sync. It should decide whether to
> actually perform a syncscan using the logic from initscan(), and then
> it should store a phs_syncscan flag into the ParallelHeapScanDesc.
> heap_beginscan_internal should set rs_syncscan based on phs_syncscan,
> regardless of anything else.
>
> I think heap_parallel_rescan() is an unworkable API. When rescanning
> a synchronized scan, the existing logic keeps the original start-block
> so that the scan's results are reproducible,
> scan position since we're presumably out of step with other backends.
> This isn't going to work at all with this API. I don't think you can
> swap out the ParallelHeapScanDesc for another one once you've
> installed it;
> This is obviously going to present some design complications for the
> as-yet-uncommitted code to push down PARAM_EXEC parameters, because if
> the new value takes more bytes to store than the old value, there
> won't be room to update the existing DSM in place.
On Fri, Oct 2, 2015 at 11:44 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Oct 1, 2015 at 7:41 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> Does heap_parallelscan_nextpage really need a pscan_finished output >> parameter, or can it just return InvalidBlockNumber to indicate end of >> scan? I think the latter can be done and would be cleaner. > > I think we can do that way as well, however we have to keep the check > of page == InvalidBlockNumber at all the callers to indicate finish > of scan which made me write the function as it exists in patch. However, > I don't mind changing it the way you have suggested if you feel that would > be cleaner. I think it would. I mean, you just end up testing the other thing instead. >> I think that heap_parallelscan_initialize() should taken an >> additional Boolean argument, allow_sync. It should decide whether to >> actually perform a syncscan using the logic from initscan(), and then >> it should store a phs_syncscan flag into the ParallelHeapScanDesc. >> heap_beginscan_internal should set rs_syncscan based on phs_syncscan, >> regardless of anything else. > > I think this will ensure that any future change in this area won't break the > guarantee for sync scans for parallel workers, is that the reason you > prefer to implement this function in the way suggested by you? Basically. It seems pretty fragile the way you have it now. >> I think heap_parallel_rescan() is an unworkable API. When rescanning >> a synchronized scan, the existing logic keeps the original start-block >> so that the scan's results are reproducible, > > It seems from the code comments in initscan, the reason for keeping > previous startblock is to allow rewinding the cursor which doesn't hold for > parallel scan. We might or might not want to support such cases with > parallel query, but even if we want to there is a way we can do with > current logic (as mentioned in one of my replies below). You don't need to rewind a cursor; you just need to restart the scan. So for example if we were on the inner side of a NestLoop, this would be a real issue. >> but no longer reports the >> scan position since we're presumably out of step with other backends. > > Is it true for all form of rescans or are you talking about some > case (like SampleScan) in particular? As per my reading of code > (and verified by debugging that code path), it doesn't seem to be true > for rescan in case of seqscan. I think it is: if (keep_startblock) { /* * When rescanning, we want to keep the previous startblock setting, *so that rewinding a cursor doesn't generate surprising results. * Reset the active syncscan setting, though. */ scan->rs_syncscan = (allow_sync && synchronize_seqscans); } >> This isn't going to work at all with this API. I don't think you can >> swap out the ParallelHeapScanDesc for another one once you've >> installed it; >> > > Sure, but if we really need some such parameters like startblock position, > then we can preserve those in ScanDesc. Sure, we could transfer the information out of the ParallelHeapScanDesc and then transfer it back into the new one, but I have a hard time thinking that's a good design. > PARAM_EXEC parameters will be used to support initPlan in parallel query (as > it is done in the initial patch), so it seems to me this is the main > downside of > this approach. I think rather than trying to come up with various possible > solutions for this problem, lets prohibit sync scans with parallel query if > you > see some problem with the suggestions made by me above. Do you see any > main use case getting hit due to non support of sync scans with > parallel seq. scan? Yes. Synchronized scans are particularly important with large tables, and those are the kind you're likely to want to use a parallel sequential scan on. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Oct 2, 2015 at 11:44 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Thu, Oct 1, 2015 at 7:41 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >> Does heap_parallelscan_nextpage really need a pscan_finished output
> >> parameter, or can it just return InvalidBlockNumber to indicate end of
> >> scan? I think the latter can be done and would be cleaner.
> >
> > I think we can do that way as well, however we have to keep the check
> > of page == InvalidBlockNumber at all the callers to indicate finish
> > of scan which made me write the function as it exists in patch. However,
> > I don't mind changing it the way you have suggested if you feel that would
> > be cleaner.
>
> I think it would. I mean, you just end up testing the other thing instead.
>
> >> I think that heap_parallelscan_initialize() should taken an
> >> additional Boolean argument, allow_sync. It should decide whether to
> >> actually perform a syncscan using the logic from initscan(), and then
> >> it should store a phs_syncscan flag into the ParallelHeapScanDesc.
> >> heap_beginscan_internal should set rs_syncscan based on phs_syncscan,
> >> regardless of anything else.
> >
> > I think this will ensure that any future change in this area won't break the
> > guarantee for sync scans for parallel workers, is that the reason you
> > prefer to implement this function in the way suggested by you?
>
> Basically. It seems pretty fragile the way you have it now.
>
> >> I think heap_parallel_rescan() is an unworkable API. When rescanning
> >> a synchronized scan, the existing logic keeps the original start-block
> >> so that the scan's results are reproducible,
> >
> > It seems from the code comments in initscan, the reason for keeping
> > previous startblock is to allow rewinding the cursor which doesn't hold for
> > parallel scan. We might or might not want to support such cases with
> > parallel query, but even if we want to there is a way we can do with
> > current logic (as mentioned in one of my replies below).
>
> You don't need to rewind a cursor; you just need to restart the scan.
> So for example if we were on the inner side of a NestLoop, this would
> be a real issue.
>
> >> but no longer reports the
> >> scan position since we're presumably out of step with other backends.
> >
> > Is it true for all form of rescans or are you talking about some
> > case (like SampleScan) in particular? As per my reading of code
> > (and verified by debugging that code path), it doesn't seem to be true
> > for rescan in case of seqscan.
>
> I think it is:
>
> if (keep_startblock)
> {
> /*
> * When rescanning, we want to keep the previous startblock setting,
> * so that rewinding a cursor doesn't generate surprising results.
> * Reset the active syncscan setting, though.
> */
> scan->rs_syncscan = (allow_sync && synchronize_seqscans);
> }
>
> >> This isn't going to work at all with this API. I don't think you can
> >> swap out the ParallelHeapScanDesc for another one once you've
> >> installed it;
> >>
> >
> > Sure, but if we really need some such parameters like startblock position,
> > then we can preserve those in ScanDesc.
>
> Sure, we could transfer the information out of the
> ParallelHeapScanDesc and then transfer it back into the new one, but I
> have a hard time thinking that's a good design.
>
>
> On Sat, Oct 3, 2015 at 11:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > On Fri, Oct 2, 2015 at 11:44 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > > On Thu, Oct 1, 2015 at 7:41 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> > >> Does heap_parallelscan_nextpage really need a pscan_finished output
> > >> parameter, or can it just return InvalidBlockNumber to indicate end of
> > >> scan? I think the latter can be done and would be cleaner.
> > >
> > > I think we can do that way as well, however we have to keep the check
> > > of page == InvalidBlockNumber at all the callers to indicate finish
> > > of scan which made me write the function as it exists in patch. However,
> > > I don't mind changing it the way you have suggested if you feel that would
> > > be cleaner.
> >
> > I think it would. I mean, you just end up testing the other thing instead.
> >
>
> No issues, will change in next version of patch.
>
> > >> I think that heap_parallelscan_initialize() should taken an
> > >> additional Boolean argument, allow_sync. It should decide whether to
> > >> actually perform a syncscan using the logic from initscan(), and then
> > >> it should store a phs_syncscan flag into the ParallelHeapScanDesc.
> > >> heap_beginscan_internal should set rs_syncscan based on phs_syncscan,
> > >> regardless of anything else.
> > >
> > > I think this will ensure that any future change in this area won't break the
> > > guarantee for sync scans for parallel workers, is that the reason you
> > > prefer to implement this function in the way suggested by you?
> >
> > Basically. It seems pretty fragile the way you have it now.
> >
>
> Okay, in that case I will change it as per your suggestion.
>
> > >> I think heap_parallel_rescan() is an unworkable API. When rescanning
> > >> a synchronized scan, the existing logic keeps the original start-block
> > >> so that the scan's results are reproducible,
> > >
> > > It seems from the code comments in initscan, the reason for keeping
> > > previous startblock is to allow rewinding the cursor which doesn't hold for
> > > parallel scan. We might or might not want to support such cases with
> > > parallel query, but even if we want to there is a way we can do with
> > > current logic (as mentioned in one of my replies below).
> >
> > You don't need to rewind a cursor; you just need to restart the scan.
> > So for example if we were on the inner side of a NestLoop, this would
> > be a real issue.
> >
>
> Sorry, but I am not able to see the exact issue you have in mind for NestLoop,
> if we don't preserve the start block position for parallel scan.
Attachment
On Sat, Sep 26, 2015 at 04:09:12PM -0400, Robert Haas wrote: > +/*------------------------------------------------------------------------- > + * datumSerialize > + * > + * Serialize a possibly-NULL datum into caller-provided storage. > +void > +datumSerialize(Datum value, bool isnull, bool typByVal, int typLen, > + char **start_address) > +{ > + int header; > + > + /* Write header word. */ > + if (isnull) > + header = -2; > + else if (typByVal) > + header = -1; > + else > + header = datumGetSize(value, typByVal, typLen); > + memcpy(*start_address, &header, sizeof(int)); > + *start_address += sizeof(int); > + > + /* If not null, write payload bytes. */ > + if (!isnull) > + { > + if (typByVal) > + { > + memcpy(*start_address, &value, sizeof(Datum)); > + *start_address += sizeof(Datum); > + } > + else > + { > + memcpy(*start_address, DatumGetPointer(value), header); > + *start_address += header; > + } > + } > +} I see no mention in this thread of varatt_indirect, but I anticipated datumSerialize() reacting to it the same way datumCopy() reacts. If datumSerialize() can get away without doing so, why is that?
On Mon, Oct 5, 2015 at 11:20 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > For now, I have fixed this by not preserving the startblock incase of rescan > for parallel scan. Note that, I have created a separate patch > (parallel_seqscan_heaprescan_v1.patch) for support of rescan (for parallel > scan). while testing parallel seqscan, My colleague Jing Wang has found a problem in parallel_seqscan_heapscan_v2.patch. In function initscan, the allow_sync flag is set to false as the number of pages in the table are less than NBuffers/4. if (!RelationUsesLocalBuffers(scan->rs_rd) && scan->rs_nblocks > NBuffers / 4) As allow_sync flag is false, the function heap_parallelscan_initialize_startblock is not called in initscan function to initialize the parallel_scan->phs_cblock parameter. Because of this reason while getting the next page in heap_parallelscan_nextpage, it returns InvalidBlockNumber, thus it ends the scan without returning the results. Regards, Hari Babu Fujitsu Australia
>
> On Mon, Oct 5, 2015 at 11:20 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > For now, I have fixed this by not preserving the startblock incase of rescan
> > for parallel scan. Note that, I have created a separate patch
> > (parallel_seqscan_heaprescan_v1.patch) for support of rescan (for parallel
> > scan).
>
> while testing parallel seqscan, My colleague Jing Wang has found a problem in
> parallel_seqscan_heapscan_v2.patch.
>
> In function initscan, the allow_sync flag is set to false as the
> number of pages in the
> table are less than NBuffers/4.
>
> if (!RelationUsesLocalBuffers(scan->rs_rd) &&
> scan->rs_nblocks > NBuffers / 4)
>
> As allow_sync flag is false, the function
> heap_parallelscan_initialize_startblock is not
> called in initscan function to initialize the
> parallel_scan->phs_cblock parameter. Because
> of this reason while getting the next page in
> heap_parallelscan_nextpage, it returns
> InvalidBlockNumber, thus it ends the scan without returning the results.
>
Attachment
On Sun, Oct 11, 2015 at 7:56 PM, Noah Misch <noah@leadboat.com> wrote: > I see no mention in this thread of varatt_indirect, but I anticipated > datumSerialize() reacting to it the same way datumCopy() reacts. If > datumSerialize() can get away without doing so, why is that? Good point. I don't think it can. Attached is a patch to fix that. This patch also includes some somewhat-related changes to plpgsql_param_fetch() upon which I would appreciate any input you can provide. plpgsql_param_fetch() assumes that it can detect whether it's being called from copyParamList() by checking whether params != estate->paramLI. I don't know why this works, but I do know that this test fails to detect the case where it's being called from SerializeParamList(), which causes failures in exec_eval_datum() as predicted. Calls from SerializeParamList() need the same treatment as calls from copyParamList() because it, too, will try to evaluate every parameter in the list. Here, I've taken the approach of making that check unconditional, which seems to work, but I'm not sure if some other approach would be better, such as adding an additional Boolean (or enum context?) argument to ParamFetchHook. I *think* that skipping this check is merely a performance optimization rather than anything that affects correctness, and bms_is_member() is pretty cheap, so perhaps the way I've done it is OK. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
>
>
> Right, it should initialize parallel scan properly even for non-synchronized
> scans. Fixed the issue in attached patch. Rebased heap rescan is
> attached as well.
>
Attachment
On Tue, Oct 13, 2015 at 2:45 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Attached is rebased patch for partial seqscan support. Review comments: - If you're going to pgindent execParallel.c, you need to add some entries to typedefs.list so it doesn't mangle the formatting. ExecParallelEstimate's parameter list is misformatted, for example. Also, I think if we're going to do this we had better extract the pgindent changes and commit those first. It's pretty distracting the way you have it. - Instead of inlining the work needed by each parallel mode in ExecParallelEstimate(), I think you should mimic the style of ExecProcNode and call a node-type specific function that is part of that node's public interface - here, ExecPartialSeqScanEstimate, perhaps. Similarly for ExecParallelInitializeDSM. Perhaps ExecPartialSeqScanInitializeDSM. - I continue to think GetParallelShmToc is the wrong approach. Instead, each time ExecParallelInitializeDSM or ExecParallelInitializeDSM calls a nodetype-specific initialized function (as described in the previous point), have it pass d->pcxt as an argument. The node can get the toc from there if it needs it. I suppose it could store a pointer to the toc in its scanstate if it needs it, but it really shouldn't. Instead, it should store a pointer to, say, the ParallelHeapScanDesc in the scanstate. It really should only care about its own shared data, so once it finds that, the toc shouldn't be needed any more. Then ExecPartialSeqScan doesn't need to look up pscan; it's already recorded in the scanstate. - ExecParallelInitializeDSMContext's new pscan_len member is 100% wrong. Individual scan nodes don't get to add stuff to that context object. They should store details like this in their own ScanState as needed. - The positioning of the new nodes in various lists doesn't seem to entirely consistent. nodes.h adds them after SampleScan which isn't terrible, though maybe immediately after SeqScan would be better, but _outNode has it right after BitmapOr and the switch in _copyObject has it somewhere else again. - Although the changes in parallelpaths.c are in a good direction, I'm pretty sure this is not yet up to scratch. I am less sure exactly what needs to be fixed, so I'll have to give some more thought to that. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Mon, Oct 12, 2015 at 11:46:08AM -0400, Robert Haas wrote: > plpgsql_param_fetch() assumes that it can detect whether it's being > called from copyParamList() by checking whether params != > estate->paramLI. I don't know why this works, but I do know that this It works because PL/pgSQL creates an unshared list whenever copyParamList() is forthcoming. (This in turn relies on intimate knowledge of how the rest of the system processes param lists.) The comments at setup_param_list() and setup_unshared_param_list() are most pertinent. > test fails to detect the case where it's being called from > SerializeParamList(), which causes failures in exec_eval_datum() as > predicted. Calls from SerializeParamList() need the same treatment as > calls from copyParamList() because it, too, will try to evaluate every > parameter in the list. Here, I've taken the approach of making that > check unconditional, which seems to work, but I'm not sure if some > other approach would be better, such as adding an additional Boolean > (or enum context?) argument to ParamFetchHook. I *think* that > skipping this check is merely a performance optimization rather than > anything that affects correctness, and bms_is_member() is pretty > cheap, so perhaps the way I've done it is OK. Like you, I don't expect bms_is_member() to be expensive relative to the task at hand. However, copyParamList() and SerializeParamList() copy non-dynamic params without calling plpgsql_param_fetch(). Given the shared param list, they will copy non-dynamic params the current query doesn't use. That cost is best avoided, not being well-bounded; consider the case of an unrelated variable containing a TOAST pointer to a 1-GiB value. One approach is to have setup_param_list() copy the paramnos pointer to a new ParamListInfoData field: Bitmapset *paramMask; /* if non-NULL, ignore params lacking a 1-bit */ Test it directly in copyParamList() and SerializeParamList(). As a bonus, that would allow use of the shared param list for more cases involving cursors. Furthermore, plpgsql_param_fetch() would never need to test paramnos. A more-general alternative is to have a distinct "paramIsUsed" callback, but I don't know how one would exploit the extra generality.
>
> On Tue, Oct 13, 2015 at 2:45 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached is rebased patch for partial seqscan support.
>
> Review comments:
>
>
> - I continue to think GetParallelShmToc is the wrong approach.
> Instead, each time ExecParallelInitializeDSM or
> ExecParallelInitializeDSM calls a nodetype-specific initialized
> function (as described in the previous point), have it pass d->pcxt as
> an argument. The node can get the toc from there if it needs it. I
> suppose it could store a pointer to the toc in its scanstate if it
> needs it, but it really shouldn't. Instead, it should store a pointer
> to, say, the ParallelHeapScanDesc in the scanstate.
>
With Regards,
Amit Kapila.
On Wed, Oct 14, 2015 at 12:30 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> - I continue to think GetParallelShmToc is the wrong approach. >> Instead, each time ExecParallelInitializeDSM or >> ExecParallelInitializeDSM calls a nodetype-specific initialized >> function (as described in the previous point), have it pass d->pcxt as >> an argument. The node can get the toc from there if it needs it. I >> suppose it could store a pointer to the toc in its scanstate if it >> needs it, but it really shouldn't. Instead, it should store a pointer >> to, say, the ParallelHeapScanDesc in the scanstate. > > How will this idea work for worker backend. Basically in worker > if want something like this to work, toc has to be passed via > QueryDesc to Estate and then we can retrieve ParallelHeapScanDesc > during PartialSeqScan initialization (ExecInitPartialSeqScan). > Do you have something else in mind? Good question. I think when the worker starts up it should call yet another planstate-walker, e.g. ExecParallelInitializeWorker, which can call nodetype-specific functions for parallel-aware nodes and give each of them a chance to access the toc and store a pointer to their parallel shared state (ParallelHeapScanDesc in this case) in their scanstate. I think this should get called from ParallelQueryMain after ExecutorStart and before ExecutorRun: ExecParallelInitializeWorker(queryDesc->planstate, toc). -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Tue, Oct 13, 2015 at 9:08 PM, Noah Misch <noah@leadboat.com> wrote: > On Mon, Oct 12, 2015 at 11:46:08AM -0400, Robert Haas wrote: >> plpgsql_param_fetch() assumes that it can detect whether it's being >> called from copyParamList() by checking whether params != >> estate->paramLI. I don't know why this works, but I do know that this > > It works because PL/pgSQL creates an unshared list whenever copyParamList() is > forthcoming. (This in turn relies on intimate knowledge of how the rest of > the system processes param lists.) The comments at setup_param_list() and > setup_unshared_param_list() are most pertinent. Thanks for the pointer. >> test fails to detect the case where it's being called from >> SerializeParamList(), which causes failures in exec_eval_datum() as >> predicted. Calls from SerializeParamList() need the same treatment as >> calls from copyParamList() because it, too, will try to evaluate every >> parameter in the list. Here, I've taken the approach of making that >> check unconditional, which seems to work, but I'm not sure if some >> other approach would be better, such as adding an additional Boolean >> (or enum context?) argument to ParamFetchHook. I *think* that >> skipping this check is merely a performance optimization rather than >> anything that affects correctness, and bms_is_member() is pretty >> cheap, so perhaps the way I've done it is OK. > > Like you, I don't expect bms_is_member() to be expensive relative to the task > at hand. However, copyParamList() and SerializeParamList() copy non-dynamic > params without calling plpgsql_param_fetch(). Given the shared param list, > they will copy non-dynamic params the current query doesn't use. That cost is > best avoided, not being well-bounded; consider the case of an unrelated > variable containing a TOAST pointer to a 1-GiB value. One approach is to have > setup_param_list() copy the paramnos pointer to a new ParamListInfoData field: > > Bitmapset *paramMask; /* if non-NULL, ignore params lacking a 1-bit */ > > Test it directly in copyParamList() and SerializeParamList(). As a bonus, > that would allow use of the shared param list for more cases involving > cursors. Furthermore, plpgsql_param_fetch() would never need to test > paramnos. A more-general alternative is to have a distinct "paramIsUsed" > callback, but I don't know how one would exploit the extra generality. I'm anxious to minimize the number of things that must be fixed in order for a stable version of parallel query to exist in our master repository, and I fear that trying to improve ParamListInfo generally could take me fairly far afield. How about adding a paramListCopyHook to ParamListInfoData? SerializeParamList() would, if it found a parameter with !OidIsValid(prm->prmtype) && param->paramFetch != NULL, call this function, which would return a new ParamListInfo to be serialized in place of the original? This wouldn't require any modification to the current plpgsql_param_fetch() at all, but the new function would steal its bms_is_member() test. Furthermore, no user of ParamListInfo other than plpgsql needs to care at all -- which, with your proposals, they would. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Oct 14, 2015 at 07:52:15PM -0400, Robert Haas wrote: > On Tue, Oct 13, 2015 at 9:08 PM, Noah Misch <noah@leadboat.com> wrote: > > On Mon, Oct 12, 2015 at 11:46:08AM -0400, Robert Haas wrote: > >> Calls from SerializeParamList() need the same treatment as > >> calls from copyParamList() because it, too, will try to evaluate every > >> parameter in the list. > > Like you, I don't expect bms_is_member() to be expensive relative to the task > > at hand. However, copyParamList() and SerializeParamList() copy non-dynamic > > params without calling plpgsql_param_fetch(). Given the shared param list, > > they will copy non-dynamic params the current query doesn't use. That cost is > > best avoided, not being well-bounded; consider the case of an unrelated > > variable containing a TOAST pointer to a 1-GiB value. One approach is to have > > setup_param_list() copy the paramnos pointer to a new ParamListInfoData field: > > > > Bitmapset *paramMask; /* if non-NULL, ignore params lacking a 1-bit */ > > > > Test it directly in copyParamList() and SerializeParamList(). As a bonus, > > that would allow use of the shared param list for more cases involving > > cursors. Furthermore, plpgsql_param_fetch() would never need to test > > paramnos. A more-general alternative is to have a distinct "paramIsUsed" > > callback, but I don't know how one would exploit the extra generality. > > I'm anxious to minimize the number of things that must be fixed in > order for a stable version of parallel query to exist in our master > repository, and I fear that trying to improve ParamListInfo generally > could take me fairly far afield. How about adding a paramListCopyHook > to ParamListInfoData? SerializeParamList() would, if it found a > parameter with !OidIsValid(prm->prmtype) && param->paramFetch != NULL, > call this function, which would return a new ParamListInfo to be > serialized in place of the original? Tests of prm->prmtype and paramLI->paramFetch appear superfluous. Given that the paramListCopyHook callback would return a complete substitute ParamListInfo, I wouldn't expect SerializeParamList() to examine the the original paramLI->params at all. If that's correct, the paramListCopyHook design sounds fine. However, its implementation will be more complex than paramMask would have been. > This wouldn't require any > modification to the current plpgsql_param_fetch() at all, but the new > function would steal its bms_is_member() test. Furthermore, no user > of ParamListInfo other than plpgsql needs to care at all -- which, > with your proposals, they would. To my knowledge, none of these approaches would compel existing users to care. They would leave paramMask or paramListCopyHook NULL and get today's behavior.
>
> On Tue, Oct 13, 2015 at 2:45 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Attached is rebased patch for partial seqscan support.
>
> Review comments:
>
> - If you're going to pgindent execParallel.c, you need to add some
> entries to typedefs.list so it doesn't mangle the formatting.
> ExecParallelEstimate's parameter list is misformatted, for example.
> Also, I think if we're going to do this we had better extract the
> pgindent changes and commit those first. It's pretty distracting the
> way you have it.
>
> - Instead of inlining the work needed by each parallel mode in
> ExecParallelEstimate(), I think you should mimic the style of
> ExecProcNode and call a node-type specific function that is part of
> that node's public interface - here, ExecPartialSeqScanEstimate,
> perhaps. Similarly for ExecParallelInitializeDSM. Perhaps
> ExecPartialSeqScanInitializeDSM.
>
> - I continue to think GetParallelShmToc is the wrong approach.
> Instead, each time ExecParallelInitializeDSM or
> ExecParallelInitializeDSM calls a nodetype-specific initialized
> function (as described in the previous point), have it pass d->pcxt as
> an argument. The node can get the toc from there if it needs it. I
> suppose it could store a pointer to the toc in its scanstate if it
> needs it, but it really shouldn't. Instead, it should store a pointer
> to, say, the ParallelHeapScanDesc in the scanstate. It really should
> only care about its own shared data, so once it finds that, the toc
> shouldn't be needed any more. Then ExecPartialSeqScan doesn't need to
> look up pscan; it's already recorded in the scanstate.
>
> - ExecParallelInitializeDSMContext's new pscan_len member is 100%
> wrong. Individual scan nodes don't get to add stuff to that context
> object. They should store details like this in their own ScanState as
> needed.
>
> - The positioning of the new nodes in various lists doesn't seem to
> entirely consistent. nodes.h adds them after SampleScan which isn't
> terrible, though maybe immediately after SeqScan would be better, but
> _outNode has it right after BitmapOr and the switch in _copyObject has
> it somewhere else again.
>
Attachment
>
> On Sun, Oct 11, 2015 at 7:56 PM, Noah Misch <noah@leadboat.com> wrote:
> > I see no mention in this thread of varatt_indirect, but I anticipated
> > datumSerialize() reacting to it the same way datumCopy() reacts. If
> > datumSerialize() can get away without doing so, why is that?
>
> Good point. I don't think it can. Attached is a patch to fix that.
> This patch also includes some somewhat-related changes to
> plpgsql_param_fetch() upon which I would appreciate any input you can
> provide.
>
> plpgsql_param_fetch() assumes that it can detect whether it's being
> called from copyParamList() by checking whether params !=
> estate->paramLI. I don't know why this works, but I do know that this
> test fails to detect the case where it's being called from
> SerializeParamList(), which causes failures in exec_eval_datum() as
> predicted. Calls from SerializeParamList() need the same treatment as
> calls from copyParamList() because it, too, will try to evaluate every
> parameter in the list.
>
On Thu, Oct 15, 2015 at 6:32 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> On Wed, Oct 14, 2015 at 3:29 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> I think this got messed up while rebasing on top of Gather node
> changes, but nonetheless, I have changed it such that PartialSeqScan
> node handling is after SeqScan.
Currently, the explain analyze of parallel seq scan plan is not showing the allocated number of workers
[1] http://www.postgresql.org/message-id/CA+TgmobhQ0_+YObMLbJexvt4QEf6XbLfUdaX1OwL-ivgaN5qxw@mail.gmail.com
Regards,
Hari Babu
Fujitsu Australia
>
>
>
> On Thu, Oct 15, 2015 at 6:32 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Wed, Oct 14, 2015 at 3:29 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> > I think this got messed up while rebasing on top of Gather node
> > changes, but nonetheless, I have changed it such that PartialSeqScan
> > node handling is after SeqScan.
>
> Currently, the explain analyze of parallel seq scan plan is not showing the allocated number of workers
> including the planned workers.I feel this information is good for users in understanding the performance
> difference that is coming with parallel seq scan. It may be missed in recent patch series. It was discussed
> in[1].
>
> Currently there is no qualification evaluation at Result and Gather nodes, because of this reason, if any
> query that contains any parallel restricted functions is not chosen for parallel scan. Because of
> this reason, there is no difference between parallel restricted and parallel unsafe functions currently.
On Thu, Oct 15, 2015 at 7:00 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Mon, Oct 12, 2015 at 9:16 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> On Sun, Oct 11, 2015 at 7:56 PM, Noah Misch <noah@leadboat.com> wrote: >> > I see no mention in this thread of varatt_indirect, but I anticipated >> > datumSerialize() reacting to it the same way datumCopy() reacts. If >> > datumSerialize() can get away without doing so, why is that? >> >> Good point. I don't think it can. Attached is a patch to fix that. >> This patch also includes some somewhat-related changes to >> plpgsql_param_fetch() upon which I would appreciate any input you can >> provide. >> >> plpgsql_param_fetch() assumes that it can detect whether it's being >> called from copyParamList() by checking whether params != >> estate->paramLI. I don't know why this works, but I do know that this >> test fails to detect the case where it's being called from >> SerializeParamList(), which causes failures in exec_eval_datum() as >> predicted. Calls from SerializeParamList() need the same treatment as >> calls from copyParamList() because it, too, will try to evaluate every >> parameter in the list. > > From what I understood by looking at code in this area, I think the check > params != estate->paramLI and code under it is required for parameters > that are setup by setup_unshared_param_list(). Now unshared params > are only created for Cursors and expressions that are passing a R/W > object pointer; for cursors we explicitly prohibit the parallel plan > generation > and I am not sure if it makes sense to generate parallel plans for > expressions > involving R/W object pointer, if we don't generate parallel plan where > expressions involve such parameters, then SerializeParamList() should not > be affected by the check mentioned by you. Is by anychance, this is > happening because you are testing by forcing gather node on top of > all kind of plans? Yeah, but I think the scenario is legitimate. When a query gets run from within PL/pgsql, parallelism is an option, at least as we have the code today. So if a Gather were present, and the query used a parameter, then you could have this issue. For example: SELECT * FROM bigtable WHERE unindexed_column = some_plpgsql_variable; So this can happen, I think, even with parallel sequential scan only, even if Gather node is not otherwise used. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Oct 15, 2015 at 1:51 AM, Noah Misch <noah@leadboat.com> wrote: > Tests of prm->prmtype and paramLI->paramFetch appear superfluous. Given that > the paramListCopyHook callback would return a complete substitute > ParamListInfo, I wouldn't expect SerializeParamList() to examine the the > original paramLI->params at all. If that's correct, the paramListCopyHook > design sounds fine. However, its implementation will be more complex than > paramMask would have been. Well, I think there are two use cases we care about. If the ParamListInfo came from Bind parameters sent via a protocol message, then it will neither have a copy method nor require one. If it came from some source that plays fancy games, like PL/pgsql, then it needs a safe way to copy the list. >> This wouldn't require any >> modification to the current plpgsql_param_fetch() at all, but the new >> function would steal its bms_is_member() test. Furthermore, no user >> of ParamListInfo other than plpgsql needs to care at all -- which, >> with your proposals, they would. > > To my knowledge, none of these approaches would compel existing users to care. > They would leave paramMask or paramListCopyHook NULL and get today's behavior. Well, looking at this proposal: Bitmapset *paramMask; /* if non-NULL, ignore params lacking a 1-bit */ I read that to imply that every consumer of ParamListInfo objects would need to account for the possibility of getting one with a non-NULL paramMask. Would it work to define this as "if non-NULL, params lacking a 1-bit may be safely ignored"? Or some other tweak that basically says that you don't need to care about this, but you can if you want to. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Oct 15, 2015 at 12:05:53PM -0400, Robert Haas wrote: > On Thu, Oct 15, 2015 at 1:51 AM, Noah Misch <noah@leadboat.com> wrote: > >> This wouldn't require any > >> modification to the current plpgsql_param_fetch() at all, but the new > >> function would steal its bms_is_member() test. Furthermore, no user > >> of ParamListInfo other than plpgsql needs to care at all -- which, > >> with your proposals, they would. > > > > To my knowledge, none of these approaches would compel existing users to care. > > They would leave paramMask or paramListCopyHook NULL and get today's behavior. > > Well, looking at this proposal: > > Bitmapset *paramMask; /* if non-NULL, ignore params lacking a 1-bit */ > > I read that to imply that every consumer of ParamListInfo objects > would need to account for the possibility of getting one with a > non-NULL paramMask. Agreed. More specifically, I had in mind for copyParamList() to check the mask while e.g. ExecEvalParamExtern() would either check nothing or merely assert that any mask included the requested parameter. It would be tricky to verify that as safe, so ... > Would it work to define this as "if non-NULL, > params lacking a 1-bit may be safely ignored"? Or some other tweak > that basically says that you don't need to care about this, but you > can if you want to. ... this is a better specification.
On Thu, Oct 15, 2015 at 11:45 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Oct 15, 2015 at 5:39 PM, Haribabu Kommi <kommi.haribabu@gmail.com> > wrote: >> >> >> >> On Thu, Oct 15, 2015 at 6:32 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > On Wed, Oct 14, 2015 at 3:29 AM, Robert Haas <robertmhaas@gmail.com> >> > wrote: >> > I think this got messed up while rebasing on top of Gather node >> > changes, but nonetheless, I have changed it such that PartialSeqScan >> > node handling is after SeqScan. >> >> Currently, the explain analyze of parallel seq scan plan is not showing >> the allocated number of workers >> including the planned workers.I feel this information is good for users in >> understanding the performance >> difference that is coming with parallel seq scan. It may be missed in >> recent patch series. It was discussed >> in[1]. >> > > I am aware of that and purposefully kept it for a consecutive patch. > There are other things as well which I have left out from this patch > and those are: > a. Early stop of executor for Rescan purpose > b. Support of pushdown for plans containing InitPlan and SubPlans > > Then there is more related work like > a. Support for prepared statements > OK. During the test with latest patch, I found a dead lock between worker and backend on relation lock. To minimize the test scenario, I changed the number of pages required to start one worker to 1 and all parallel cost parameters as zero. Backend is waiting for the tuples from workers, workers are waiting on lock of relation. Attached is the sql script that can reproduce this issue. Regards, Hari Babu Fujitsu Australia
Attachment
On Fri, Oct 16, 2015 at 2:10 PM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > On Thu, Oct 15, 2015 at 11:45 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> On Thu, Oct 15, 2015 at 5:39 PM, Haribabu Kommi <kommi.haribabu@gmail.com> >> wrote: >>> >>> >>> >>> On Thu, Oct 15, 2015 at 6:32 PM, Amit Kapila <amit.kapila16@gmail.com> >>> wrote: >>> > On Wed, Oct 14, 2015 at 3:29 AM, Robert Haas <robertmhaas@gmail.com> >>> > wrote: >>> > I think this got messed up while rebasing on top of Gather node >>> > changes, but nonetheless, I have changed it such that PartialSeqScan >>> > node handling is after SeqScan. >>> >>> Currently, the explain analyze of parallel seq scan plan is not showing >>> the allocated number of workers >>> including the planned workers.I feel this information is good for users in >>> understanding the performance >>> difference that is coming with parallel seq scan. It may be missed in >>> recent patch series. It was discussed >>> in[1]. >>> >> >> I am aware of that and purposefully kept it for a consecutive patch. >> There are other things as well which I have left out from this patch >> and those are: >> a. Early stop of executor for Rescan purpose >> b. Support of pushdown for plans containing InitPlan and SubPlans >> >> Then there is more related work like >> a. Support for prepared statements >> > > OK. > > During the test with latest patch, I found a dead lock between worker > and backend > on relation lock. To minimize the test scenario, I changed the number > of pages required > to start one worker to 1 and all parallel cost parameters as zero. > > Backend is waiting for the tuples from workers, workers are waiting on > lock of relation. > Attached is the sql script that can reproduce this issue. Some more tests that failed in similar configuration settings. 1. Table that is created under a begin statement is not visible in the worker. 2. permission problem in worker side for set role command. Regards, Hari Babu Fujitsu Australia
Attachment
On Mon, Oct 5, 2015 at 8:20 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > [ new patch for heapam.c changes ] I went over this in a fair amount of detail and reworked it somewhat. The result is attached as parallel-heapscan-revised.patch. I think the way I did this is a bit cleaner than what you had, although it's basically the same thing. There are fewer changes to initscan, and we don't need one function to initialize the starting block that must be called in each worker and then another one to get the next block, and generally the changes are a bit more localized. I also went over the comments and, I think, improved them. I tweaked the logic for reporting the starting scan position as the last position report; I think the way you had it the last report would be for one block earlier. I'm pretty happy with this version and hope to commit it soon. There's a second patch attached here as well, parallel-relaunch.patch, which makes it possible to relaunch workers with the same parallel context. Currently, after you WaitForParallelWorkersToFinish(), you must proceed without fail to DestroyParallelContext(). With this rather simple patch, you have the option to instead go back and again LaunchParallelWorkers(), which is nice because it avoids the overhead of setting up a new DSM and filling it with all of your transaction state a second time. I'd like to commit this as well, and I think we should revise execParallel.c to use it. Finally, I've attached some test code in parallel-dummy.patch. This demonstrates how the code in 0001 and 0002 can be used. It scans a relation, counts the tuples, and then gratuitously rescans it and counts the tuples again. This shows that rescanning works and that the syncscan position gets left in the right place. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
>
> On Thu, Oct 15, 2015 at 11:45 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Thu, Oct 15, 2015 at 5:39 PM, Haribabu Kommi <kommi.haribabu@gmail.com>
> > wrote:
> >>
> >
> > I am aware of that and purposefully kept it for a consecutive patch.
> > There are other things as well which I have left out from this patch
> > and those are:
> > a. Early stop of executor for Rescan purpose
> > b. Support of pushdown for plans containing InitPlan and SubPlans
> >
> > Then there is more related work like
> > a. Support for prepared statements
> >
>
> OK.
>
> During the test with latest patch, I found a dead lock between worker
> and backend
> on relation lock. To minimize the test scenario, I changed the number
> of pages required
> to start one worker to 1 and all parallel cost parameters as zero.
>
> Backend is waiting for the tuples from workers, workers are waiting on
> lock of relation.
>
> On Thu, Oct 15, 2015 at 7:00 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > From what I understood by looking at code in this area, I think the check
> > params != estate->paramLI and code under it is required for parameters
> > that are setup by setup_unshared_param_list(). Now unshared params
> > are only created for Cursors and expressions that are passing a R/W
> > object pointer; for cursors we explicitly prohibit the parallel plan
> > generation
> > and I am not sure if it makes sense to generate parallel plans for
> > expressions
> > involving R/W object pointer, if we don't generate parallel plan where
> > expressions involve such parameters, then SerializeParamList() should not
> > be affected by the check mentioned by you. Is by anychance, this is
> > happening because you are testing by forcing gather node on top of
> > all kind of plans?
>
> Yeah, but I think the scenario is legitimate. When a query gets run
> from within PL/pgsql, parallelism is an option, at least as we have
> the code today. So if a Gather were present, and the query used a
> parameter, then you could have this issue. For example:
>
> SELECT * FROM bigtable WHERE unindexed_column = some_plpgsql_variable;
>
>
> On Mon, Oct 5, 2015 at 8:20 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ new patch for heapam.c changes ]
>
> I went over this in a fair amount of detail and reworked it somewhat.
> The result is attached as parallel-heapscan-revised.patch. I think
> the way I did this is a bit cleaner than what you had, although it's
> basically the same thing. There are fewer changes to initscan, and we
> don't need one function to initialize the starting block that must be
> called in each worker and then another one to get the next block, and
> generally the changes are a bit more localized. I also went over the
> comments and, I think, improved them. I tweaked the logic for
> reporting the starting scan position as the last position report; I
> think the way you had it the last report would be for one block
> earlier. I'm pretty happy with this version and hope to commit it
> soon.
>
On Fri, Oct 16, 2015 at 7:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I think due to above changes it will report sync location on each page > scan, don't we want to report it once at end of scan? I think reporting for each page is correct. Isn't that what the non-parallel case does? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Oct 15, 2015 at 11:38 PM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: > Some more tests that failed in similar configuration settings. > 1. Table that is created under a begin statement is not visible in the worker. > 2. permission problem in worker side for set role command. The second problem, too, I have already posted a bug fix for, on a thread which also contains a whole bunch of other bug fixes. I'll get those committed today so you can avoid wasting time finding bugs I've already found and fixed. Thanks for testing. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Oct 16, 2015 at 7:42 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I think due to above changes it will report sync location on each page
> > scan, don't we want to report it once at end of scan?
>
> I think reporting for each page is correct. Isn't that what the
> non-parallel case does?
>
Yes, sorry I got confused.
On Fri, Oct 16, 2015 at 2:29 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Yeah, but I think the scenario is legitimate. When a query gets run >> from within PL/pgsql, parallelism is an option, at least as we have >> the code today. So if a Gather were present, and the query used a >> parameter, then you could have this issue. For example: >> >> SELECT * FROM bigtable WHERE unindexed_column = some_plpgsql_variable; >> > > I don't think for such statements the control flow will set up an unshared > param list. I have tried couple of such statements [1] and found that > always such parameters are set up by setup_param_list(). I think there > are only two possibilities which could lead to setting up of unshared > params: > > 1. Usage of cursors - This is already prohibited for parallel-mode. > 2. Usage of read-write-param - This only happens for expressions like > x := array_append(x, foo) (Refer exec_check_rw_parameter()). Read-write > params are not used for SQL statements. So this also won't be used for > parallel-mode > > There is a chance that I might be missing some case where unshared > params will be required for parallel-mode (as of today), but if not then > I think we can live without current changes. *shrug* The gather-test stuff isn't failing for no reason. Either PL/pgsql shouldn't be passing CURSOR_OPT_PARALLEL_OK, or having a parallel plan get generated there should work. There's not a third option. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Oct 15, 2015 at 04:30:01PM +0530, Amit Kapila wrote: > On Mon, Oct 12, 2015 at 9:16 PM, Robert Haas <robertmhaas@gmail.com> wrote: > > plpgsql_param_fetch() assumes that it can detect whether it's being > > called from copyParamList() by checking whether params != > > estate->paramLI. I don't know why this works, but I do know that this > > test fails to detect the case where it's being called from > > SerializeParamList(), which causes failures in exec_eval_datum() as > > predicted. Calls from SerializeParamList() need the same treatment as > > calls from copyParamList() because it, too, will try to evaluate every > > parameter in the list. > > From what I understood by looking at code in this area, I think the check > params != estate->paramLI and code under it is required for parameters > that are setup by setup_unshared_param_list(). Now unshared params > are only created for Cursors and expressions that are passing a R/W > object pointer; for cursors we explicitly prohibit the parallel > plan generation > and I am not sure if it makes sense to generate parallel plans for > expressions > involving R/W object pointer, if we don't generate parallel plan where > expressions involve such parameters, then SerializeParamList() should not > be affected by the check mentioned by you. The trouble comes from the opposite direction. A setup_unshared_param_list() list is fine under today's code, but a shared param list needs more help. To say it another way, parallel queries that use the shared estate->paramLI need, among other help, the logic now guarded by "params != estate->paramLI".
>
> On Thu, Oct 15, 2015 at 04:30:01PM +0530, Amit Kapila wrote:
> > On Mon, Oct 12, 2015 at 9:16 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> > > plpgsql_param_fetch() assumes that it can detect whether it's being
> > > called from copyParamList() by checking whether params !=
> > > estate->paramLI. I don't know why this works, but I do know that this
> > > test fails to detect the case where it's being called from
> > > SerializeParamList(), which causes failures in exec_eval_datum() as
> > > predicted. Calls from SerializeParamList() need the same treatment as
> > > calls from copyParamList() because it, too, will try to evaluate every
> > > parameter in the list.
> >
> > From what I understood by looking at code in this area, I think the check
> > params != estate->paramLI and code under it is required for parameters
> > that are setup by setup_unshared_param_list(). Now unshared params
> > are only created for Cursors and expressions that are passing a R/W
> > object pointer; for cursors we explicitly prohibit the parallel
> > plan generation
> > and I am not sure if it makes sense to generate parallel plans for
> > expressions
> > involving R/W object pointer, if we don't generate parallel plan where
> > expressions involve such parameters, then SerializeParamList() should not
> > be affected by the check mentioned by you.
>
> The trouble comes from the opposite direction. A setup_unshared_param_list()
> list is fine under today's code, but a shared param list needs more help. To
> say it another way, parallel queries that use the shared estate->paramLI need,
> among other help, the logic now guarded by "params != estate->paramLI".
>
> On Fri, Oct 16, 2015 at 2:29 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> Yeah, but I think the scenario is legitimate. When a query gets run
> >> from within PL/pgsql, parallelism is an option, at least as we have
> >> the code today. So if a Gather were present, and the query used a
> >> parameter, then you could have this issue. For example:
> >>
> >> SELECT * FROM bigtable WHERE unindexed_column = some_plpgsql_variable;
> >>
> >
> > I don't think for such statements the control flow will set up an unshared
> > param list. I have tried couple of such statements [1] and found that
> > always such parameters are set up by setup_param_list(). I think there
> > are only two possibilities which could lead to setting up of unshared
> > params:
> >
> > 1. Usage of cursors - This is already prohibited for parallel-mode.
> > 2. Usage of read-write-param - This only happens for expressions like
> > x := array_append(x, foo) (Refer exec_check_rw_parameter()). Read-write
> > params are not used for SQL statements. So this also won't be used for
> > parallel-mode
> >
> > There is a chance that I might be missing some case where unshared
> > params will be required for parallel-mode (as of today), but if not then
> > I think we can live without current changes.
>
> *shrug*
>
> The gather-test stuff isn't failing for no reason. Either PL/pgsql
> shouldn't be passing CURSOR_OPT_PARALLEL_OK, or having a parallel plan
> get generated there should work. There's not a third option.
>
static int
exec_run_select(PLpgSQL_execstate *estate,
PLpgSQL_expr *expr, long maxtuples, Portal *portalP,
bool parallelOK)
{
ParamListInfo paramLI;
int rc;
/*
* On the first call for this expression generate the plan
*/
if (expr->plan == NULL)
exec_prepare_plan(estate, expr, parallelOK ?
CURSOR_OPT_PARALLEL_OK : 0);
/*
* If a portal was requested, put the query into the portal
*/
if (portalP != NULL)
{
/*
* Set up short-lived ParamListInfo
*/
paramLI = setup_unshared_param_list(estate, expr);
*portalP = SPI_cursor_open_with_paramlist(NULL, expr->plan,
paramLI,
estate->readonly_func);
if (stmt->query != NULL)
{
/* static query */
exec_run_select(estate, stmt->query, 0, &portal, true);
>
> Agreed and on looking at code, I think in below code, if we pass
> parallelOK as true for the cases where Portal is non-null, such a
> problem could happen.
>
>
> static int
>
> exec_run_select(PLpgSQL_execstate *estate,
>
> PLpgSQL_expr *expr, long maxtuples, Portal *portalP,
>
> bool parallelOK)
>
> {
>
> ParamListInfo paramLI;
>
> int rc;
>
>
> /*
>
> * On the first call for this expression generate the plan
>
> */
>
> if (expr->plan == NULL)
>
> exec_prepare_plan(estate, expr, parallelOK ?
>
> CURSOR_OPT_PARALLEL_OK : 0);
>
>
> /*
>
> * If a portal was requested, put the query into the portal
>
> */
>
> if (portalP != NULL)
>
> {
>
> /*
>
> * Set up short-lived ParamListInfo
>
> */
>
> paramLI = setup_unshared_param_list(estate, expr);
>
>
> *portalP = SPI_cursor_open_with_paramlist(NULL, expr->plan,
>
> paramLI,
>
> estate->readonly_func);
>
>
>
>
> and one such case is
>
> exec_stmt_return_query()
> {
> ..
>
> if (stmt->query != NULL)
>
> {
>
> /* static query */
>
> exec_run_select(estate, stmt->query, 0, &portal, true);
>
> ..
> }
>
>
> In this function we are using controlled fetch mechanism (count as 50) to
> fetch the tuples which we initially thought of not supporting for parallelism,
> as such a mechanism is not built for parallel workers and that is the
> reason we want to prohibit cases where ever cursor is getting created.
>
On Sat, Oct 17, 2015 at 11:00:57AM +0530, Amit Kapila wrote: > On Sat, Oct 17, 2015 at 6:15 AM, Noah Misch <noah@leadboat.com> wrote: > > On Thu, Oct 15, 2015 at 04:30:01PM +0530, Amit Kapila wrote: > > > On Mon, Oct 12, 2015 at 9:16 PM, Robert Haas <robertmhaas@gmail.com> wrote: > > > > plpgsql_param_fetch() assumes that it can detect whether it's being > > > > called from copyParamList() by checking whether params != > > > > estate->paramLI. I don't know why this works, but I do know that this > > > > test fails to detect the case where it's being called from > > > > SerializeParamList(), which causes failures in exec_eval_datum() as > > > > predicted. Calls from SerializeParamList() need the same treatment as > > > > calls from copyParamList() because it, too, will try to evaluate every > > > > parameter in the list. > > > > > > From what I understood by looking at code in this area, I think the > check > > > params != estate->paramLI and code under it is required for parameters > > > that are setup by setup_unshared_param_list(). Now unshared params > > > are only created for Cursors and expressions that are passing a R/W > > > object pointer; for cursors we explicitly prohibit the parallel > > > plan generation > > > and I am not sure if it makes sense to generate parallel plans for > > > expressions > > > involving R/W object pointer, if we don't generate parallel plan where > > > expressions involve such parameters, then SerializeParamList() should > not > > > be affected by the check mentioned by you. > > > > The trouble comes from the opposite direction. A setup_unshared_param_list() > > list is fine under today's code, but a shared param list needs more help. To > > say it another way, parallel queries that use the shared estate->paramLI need, > > among other help, the logic now guarded by "params != estate->paramLI". > > > > Why would a parallel query need such a logic, that logic is needed mainly > for cursor with params and we don't want a parallelize such cases? This is not about mixing cursors with parallelism. Cursors get special treatment because each cursor copies its param list. Parallel query also copies (more precisely, serializes) its param list. You need certain logic for every param list subject to being copied. If PostgreSQL had no concept of cursors, we'd be writing that same logic from scratch for parallel query.
>
> On Sat, Oct 17, 2015 at 11:00:57AM +0530, Amit Kapila wrote:
> > On Sat, Oct 17, 2015 at 6:15 AM, Noah Misch <noah@leadboat.com> wrote:
> > > On Thu, Oct 15, 2015 at 04:30:01PM +0530, Amit Kapila wrote:
> > > > On Mon, Oct 12, 2015 at 9:16 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> > > > > plpgsql_param_fetch() assumes that it can detect whether it's being
> > > > > called from copyParamList() by checking whether params !=
> > > > > estate->paramLI. I don't know why this works, but I do know that this
> > > > > test fails to detect the case where it's being called from
> > > > > SerializeParamList(), which causes failures in exec_eval_datum() as
> > > > > predicted. Calls from SerializeParamList() need the same treatment as
> > > > > calls from copyParamList() because it, too, will try to evaluate every
> > > > > parameter in the list.
> > > >
> > > > From what I understood by looking at code in this area, I think the
> > check
> > > > params != estate->paramLI and code under it is required for parameters
> > > > that are setup by setup_unshared_param_list(). Now unshared params
> > > > are only created for Cursors and expressions that are passing a R/W
> > > > object pointer; for cursors we explicitly prohibit the parallel
> > > > plan generation
> > > > and I am not sure if it makes sense to generate parallel plans for
> > > > expressions
> > > > involving R/W object pointer, if we don't generate parallel plan where
> > > > expressions involve such parameters, then SerializeParamList() should
> > not
> > > > be affected by the check mentioned by you.
> > >
> > > The trouble comes from the opposite direction. A setup_unshared_param_list()
> > > list is fine under today's code, but a shared param list needs more help. To
> > > say it another way, parallel queries that use the shared estate->paramLI need,
> > > among other help, the logic now guarded by "params != estate->paramLI".
> > >
> >
> > Why would a parallel query need such a logic, that logic is needed mainly
> > for cursor with params and we don't want a parallelize such cases?
>
> This is not about mixing cursors with parallelism. Cursors get special
> treatment because each cursor copies its param list. Parallel query also
> copies (more precisely, serializes) its param list. You need certain logic
> for every param list subject to being copied.
>
On Sat, Oct 17, 2015 at 2:44 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I am not denying from that fact, the point I wanted to convey here is that > the logic guarded by "params != estate->paramLI" in plpgsql_param_fetch > is only needed if cursors are in use otherwise we won't need them even > for parallel query. Well, I think what Noah and are trying to explain is that this is not true. The problem is that, even if there are no cursors anywhere in the picture, there might be some variable in the param list that is not used by the parallel query but which, if evaluated, leads to an error. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Oct 17, 2015 at 2:15 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Agreed and on looking at code, I think in below code, if we pass > parallelOK as true for the cases where Portal is non-null, such a > problem could happen. > > and one such case is > > exec_stmt_return_query() > { > .. > > if (stmt->query != NULL) > > { > > /* static query */ > > exec_run_select(estate, stmt->query, 0, &portal, true); > > .. > } > > > In this function we are using controlled fetch mechanism (count as 50) to > fetch the tuples which we initially thought of not supporting for > parallelism, > as such a mechanism is not built for parallel workers and that is the > reason we want to prohibit cases where ever cursor is getting created. > > Do we want to support parallelism for this case on the basis that this API > will eventually fetch all the tuples by using controlled fetch mechanism? That was my idea when I made that change, but I think it's not going to work out well given the way the rest of the code works. Possibly that should be reverted for now, but maybe only after testing it. It's worth noting that, as of commit bfc78d7196eb28cd4e3d6c24f7e607bacecf1129, the consequences of invoking the executor with a fetch count have been greatly reduced. Previously, the assumption was that doing that was broken, and if you did it you got to keep both pieces. But that commit rejiggered things so that your parallel plan just gets run serially in that case. That might not be great from a performance perspective, but it beats undefined behavior by a wide margin. So I suspect that there are some decisions about where to pass CURSOR_OPT_PARALLEL_OK that need to be revisited in the light of that change. I haven't had time to do that yet, but we should do it as soon as we get time. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Sat, Oct 17, 2015 at 2:44 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I am not denying from that fact, the point I wanted to convey here is that
> > the logic guarded by "params != estate->paramLI" in plpgsql_param_fetch
> > is only needed if cursors are in use otherwise we won't need them even
> > for parallel query.
>
> Well, I think what Noah and are trying to explain is that this is not
> true. The problem is that, even if there are no cursors anywhere in
> the picture, there might be some variable in the param list that is
> not used by the parallel query but which, if evaluated, leads to an
> error.
>
I understand what Noah wants to say, it's just that I am not able to see
>
> On Mon, Oct 5, 2015 at 8:20 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > [ new patch for heapam.c changes ]
>
> There's a second patch attached here as well, parallel-relaunch.patch,
> which makes it possible to relaunch workers with the same parallel
> context. Currently, after you WaitForParallelWorkersToFinish(), you
> must proceed without fail to DestroyParallelContext(). With this
> rather simple patch, you have the option to instead go back and again
> LaunchParallelWorkers(), which is nice because it avoids the overhead
> of setting up a new DSM and filling it with all of your transaction
> state a second time. I'd like to commit this as well, and I think we
> should revise execParallel.c to use it.
>
Attachment
On Tue, Oct 13, 2015 at 5:59 PM, Robert Haas <robertmhaas@gmail.com> wrote: > - Although the changes in parallelpaths.c are in a good direction, I'm > pretty sure this is not yet up to scratch. I am less sure exactly > what needs to be fixed, so I'll have to give some more thought to > that. Please find attached a proposed set of changes that I think are better. These changes compute a consider_parallel flag for each RelOptInfo, which is true if it's a non-temporary relation whose baserestrictinfo references no PARAM_EXEC parameters, sublinks, or parallel-restricted functions. Actually, I made an effort to set the flag correctly even for baserels other than plain tables, and for joinrels, though we don't technically need that stuff until we get to the point of pushing joins beneath Gather nodes. When we get there, it will be important - any joinrel for which consider_parallel = false needn't even try to generate parallel paths, while if consider_parallel = true then we can consider it, if the costing makes sense. The advantage of this is that the logic is centralized. If we have parallel seq scan and also, say, parallel bitmap heap scan, your approach would require that we duplicate the logic to check for parallel-restricted functions for each path generation function. By caching it in the RelOptInfo, we don't have to do that. The function you wrote to generate parallel paths can just check the flag; if it's false, return without generating any paths. If it's true, then parallel paths can be considered. Ultimately, I think that each RelOptInfo should have a new List * member containing a list of partial paths for that relation. For a baserel, we generate a partial path (e.g. Partial Seq Scan). Then, we can consider turning each partial path into a complete path by pushing a Gather path on top of it. For a joinrel, we can consider generating a partial hash join or partial nest loop path by taking an outer partial path and an ordinary inner path and putting the appropriate path on top. In theory it would also be correct to generate merge join paths this way, but it's difficult to believe that such a plan would ever be anything but a disaster. These can then be used to generate a complete path by putting a Gather node on top of them, or they can bubble up to the next level of the join tree in the same way. However, I think for the first version of this we can keep it simple: if the consider_parallel flag is set on a relation, consider Gather -> Partial Seq Scan. If not, forget it. Thoughts? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
On Thu, Oct 15, 2015 at 8:23 PM, Noah Misch <noah@leadboat.com> wrote: > Agreed. More specifically, I had in mind for copyParamList() to check the > mask while e.g. ExecEvalParamExtern() would either check nothing or merely > assert that any mask included the requested parameter. It would be tricky to > verify that as safe, so ... > >> Would it work to define this as "if non-NULL, >> params lacking a 1-bit may be safely ignored"? Or some other tweak >> that basically says that you don't need to care about this, but you >> can if you want to. > > ... this is a better specification. Here's an attempt to implement that. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
Robert Haas <robertmhaas@gmail.com> writes: > On Thu, Oct 15, 2015 at 8:23 PM, Noah Misch <noah@leadboat.com> wrote: >>> Would it work to define this as "if non-NULL, >>> params lacking a 1-bit may be safely ignored"? Or some other tweak >>> that basically says that you don't need to care about this, but you >>> can if you want to. >> ... this is a better specification. > Here's an attempt to implement that. BTW, my Salesforce colleagues have been bit^H^H^Hgriping for quite some time about the performance costs associated with translating between plpgsql's internal PLpgSQL_datum-array format and the ParamListInfo representation. Maybe it's time to think about some wholesale redesign of ParamListInfo? Because TBH this patch doesn't seem like much but a kluge. It's mostly layering still-another bunch of ad-hoc restrictions on copyParamList, without removing any one of the kluges we had already. regards, tom lane
On Tue, Oct 20, 2015 at 3:04 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I have rebased the partial seq scan patch based on the above committed > patches. Now for rescanning it reuses the dsm and to achieve that we > need to ensure that workers have been completely shutdown and then > reinitializes the dsm. To ensure complete shutdown of workers, current > function WaitForParallelWorkersToFinish is not sufficient as that just > waits for the last message to receive from worker backend, so I have > written a new function WaitForParallelWorkersToDie. Also on receiving > 'X' message in HandleParallelMessage, it just frees the worker handle > without ensuring if the worker is died due to which later it will be > difficult > to even find whether worker is died or not, so I have removed that code > from HandleParallelMessage. Another change is that after receiving last > tuple in Gather node, it just shutdown down the workers without > destroying the dsm. + /* + * We can't finish transaction commit or abort until all of the + * workers are dead. This means, in particular, that we can't respond + * to interrupts at this stage. + */ + HOLD_INTERRUPTS(); + status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle); + RESUME_INTERRUPTS(); These comments are correct when this code is called from DestroyParallelContext(), but they're flat wrong when called from ReinitializeParallelDSM(). I suggest moving the comment back to DestroyParallelContext and following it with this: HOLD_INTERRUPTS(); WaitForParallelWorkersToDie(); RESUME_INTERRUPTS(); Then ditch the HOLD/RESUME interrupts in WaitForParallelWorkersToDie() itself. This hunk is a problem: case 'X': /* Terminate, indicating clean exit */ { - pfree(pcxt->worker[i].bgwhandle); pfree(pcxt->worker[i].error_mqh); - pcxt->worker[i].bgwhandle = NULL; pcxt->worker[i].error_mqh= NULL; break; } If you do that on receipt of the 'X' message, then DestroyParallelContext() might SIGTERM a worker that has supposedly exited cleanly. That seems bad. I think maybe the solution is to make DestroyParallelContext() terminate the worker only if pcxt->worker[i].error_mqh != NULL. So make error_mqh == NULL mean a clean loss of a worker: either we couldn't register it, or it exited cleanly. And bgwhandle == NULL would mean it's actually gone. It makes sense to have ExecShutdownGather and ExecShutdownGatherWorkers, but couldn't the former call the latter instead of duplicating the code? I think ReInitialize should be capitalized as Reinitialize throughout. ExecParallelReInitializeTupleQueues is almost a cut-and-paste duplicate of ExecParallelSetupTupleQueues. Please refactor this to avoid duplication - e.g. change ExecParallelSetupTupleQueues(ParallelContext *pcxt) to take a second argument bool reinit. ExecParallelReInitializeTupleQueues can just do ExecParallelSetupTupleQueues(pxct, true). -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Oct 23, 2015 at 12:31 AM, Tom Lane <tgl@sss.pgh.pa.us> wrote: > BTW, my Salesforce colleagues have been bit^H^H^Hgriping for quite some > time about the performance costs associated with translating between > plpgsql's internal PLpgSQL_datum-array format and the ParamListInfo > representation. Maybe it's time to think about some wholesale redesign of > ParamListInfo? Because TBH this patch doesn't seem like much but a kluge. > It's mostly layering still-another bunch of ad-hoc restrictions on > copyParamList, without removing any one of the kluges we had already. I have no objection to some kind of a redesign there, but (1) I don't think we're going to be better off doing that before getting Partial Seq Scan committed and (2) I don't think I'm the best-qualified person to do the work. With respect to the first point, despite my best efforts, this feature is going to have bugs, and getting it committed in November without a ParamListInfo redesign is surely going to be better for the overall stability of PostgreSQL and the timeliness of our release schedule than getting it committed in February with such a redesign -- never mind that this is far from the only redesign into which I could get sucked. I want to put in place some narrow fix for this issue so that I can move forward. Three alternatives have been proposed so far: (1) this, (2) the fix I coded and posted previously, which made plpgsql_param_fetch's bms_is_member test unconditional, and (3) not allowing PL/pgsql to run parallel queries. (3) sounds worse to me than either (1) or (2); I defer to others on which of (1) and (2) is preferable, or perhaps you have another proposal. On the second point, I really don't know enough about the problems with ParamListInfo to know what would be better, so I can't really help there. If you do and want to redesign it, fine, but I really need whatever you replace it with have an easy way of serializing and restoring it - be it nodeToString() and stringToNode(), SerializeParamList and RestoreParamList, or whatever. Without that, parallel query is going to have to be disabled for any query involving parameters, and that would be, uh, extremely sad. Also, FWIW, in my opinion, it would be far more useful to PostgreSQL for you to finish the work on upper planner path-ification ... an awful lot of people are waiting for that to be completed to start their own work, or are doing work that may have to be completely redone when that lands. YMMV, of course. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Tue, Oct 13, 2015 at 5:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> > - Although the changes in parallelpaths.c are in a good direction, I'm
> > pretty sure this is not yet up to scratch. I am less sure exactly
> > what needs to be fixed, so I'll have to give some more thought to
> > that.
>
> Please find attached a proposed set of changes that I think are
> better. These changes compute a consider_parallel flag for each
> RelOptInfo, which is true if it's a non-temporary relation whose
> baserestrictinfo references no PARAM_EXEC parameters, sublinks, or
> parallel-restricted functions. Actually, I made an effort to set the
> flag correctly even for baserels other than plain tables, and for
> joinrels, though we don't technically need that stuff until we get to
> the point of pushing joins beneath Gather nodes. When we get there,
> it will be important - any joinrel for which consider_parallel = false
> needn't even try to generate parallel paths, while if
> consider_parallel = true then we can consider it, if the costing makes
> sense.
>
> The advantage of this is that the logic is centralized. If we have
> parallel seq scan and also, say, parallel bitmap heap scan, your
> approach would require that we duplicate the logic to check for
> parallel-restricted functions for each path generation function.
>
> + /*
> + * We can't finish transaction commit or abort until all of the
> + * workers are dead. This means, in particular, that
> we can't respond
> + * to interrupts at this stage.
> + */
> + HOLD_INTERRUPTS();
> + status =
> WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
> + RESUME_INTERRUPTS();
>
> These comments are correct when this code is called from
> DestroyParallelContext(), but they're flat wrong when called from
> ReinitializeParallelDSM(). I suggest moving the comment back to
> DestroyParallelContext and following it with this:
>
> HOLD_INTERRUPTS();
> WaitForParallelWorkersToDie();
> RESUME_INTERRUPTS();
>
> Then ditch the HOLD/RESUME interrupts in WaitForParallelWorkersToDie() itself.
>
> This hunk is a problem:
>
> case 'X': /* Terminate,
> indicating clean exit */
> {
> - pfree(pcxt->worker[i].bgwhandle);
> pfree(pcxt->worker[i].error_mqh);
> - pcxt->worker[i].bgwhandle = NULL;
> pcxt->worker[i].error_mqh = NULL;
> break;
> }
>
> If you do that on receipt of the 'X' message, then
> DestroyParallelContext() might SIGTERM a worker that has supposedly
> exited cleanly. That seems bad. I think maybe the solution is to
> make DestroyParallelContext() terminate the worker only if
> pcxt->worker[i].error_mqh != NULL.
> clean loss of a worker: either we couldn't register it, or it exited
> cleanly. And bgwhandle == NULL would mean it's actually gone.
>
> It makes sense to have ExecShutdownGather and
> ExecShutdownGatherWorkers, but couldn't the former call the latter
> instead of duplicating the code?
>
> I think ReInitialize should be capitalized as Reinitialize throughout.
>
> ExecParallelReInitializeTupleQueues is almost a cut-and-paste
> duplicate of ExecParallelSetupTupleQueues. Please refactor this to
> avoid duplication - e.g. change
> ExecParallelSetupTupleQueues(ParallelContext *pcxt) to take a second
> argument bool reinit. ExecParallelReInitializeTupleQueues can just do
> ExecParallelSetupTupleQueues(pxct, true).
>
Changed as per suggestion.
Attachment
On Fri, Oct 23, 2015 at 3:35 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Considering parallelism at RelOptInfo level in the way as done in patch, > won't consider the RelOptInfo's for child relations in case of Append node. > Refer build_simple_rel(). Hmm, true, but what can go wrong there? The same quals apply to both, and either both are temp or neither is. > Also for cases when parallelism is not enabled like max_parallel_degree = 0, > the current way of doing could add an overhead of traversing the > baserestrictinfo without need. I think one way to avoid that would be check > that while setting parallelModeOK flag. Good idea. > Another point is that it will consider parallelism for cases where we really > can't parallelize example for foreign table, sample scan. As soon as we add the ability to push joins below Gather nodes, we will be able to parallelize that stuff if it is joined to something we can parallelize. That's why this flag is so handy. > One thing to note here is that we already have precedent of verifying qual > push down safety while path generation (during subquery path generation), > so it doesn't seem wrong to consider the same for parallel paths and it > would > minimize the cases where we need to evaluate parallelism. Mmm, yeah. >> The advantage of this is that the logic is centralized. If we have >> parallel seq scan and also, say, parallel bitmap heap scan, your >> approach would require that we duplicate the logic to check for >> parallel-restricted functions for each path generation function. > > Don't we anyway need that irrespective of caching it in RelOptInfo? > During bitmappath creation, bitmapqual could contain something > which needs to be evaluated for parallel-safety as it is built based > on index paths which inturn can be based on some join clause. As per > patch, the join clause parallel-safety is checked much later than > generation bitmappath. Yes, it's possible there could be some additional checks needed here for parameterized paths. But we're not quite there yet, so I think we can solve that problem when we get there. I have it in mind that in the future we may want a parallel_safe flag on each path, which would normally match the consider_parallel flag on the RelOptInfo but could instead be false if the path internally uses parallelism (since, currently, Gather nodes cannot be nested) or if it's got parallel-restricted parameterized quals. However, that seems like future work. > + else if (IsA(node, SubPlan) || IsA(node, SubLink) || > + IsA(node, AlternativeSubPlan) || IsA(node, Param)) > + { > + /* > + * Since we don't have the ability to push subplans down to workers > + * at present, we treat subplan references as parallel-restricted. > + */ > + if (!context->allow_restricted) > + return true; > + } > > I think it is better to do this for PARAM_EXEC paramkind, as those are > the cases where it would be subplan or initplan. Right, OK. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Oct 23, 2015 at 3:35 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Considering parallelism at RelOptInfo level in the way as done in patch,
> > won't consider the RelOptInfo's for child relations in case of Append node.
> > Refer build_simple_rel().
>
> Hmm, true, but what can go wrong there? The same quals apply to both,
> and either both are temp or neither is.
>
>
> >> The advantage of this is that the logic is centralized. If we have
> >> parallel seq scan and also, say, parallel bitmap heap scan, your
> >> approach would require that we duplicate the logic to check for
> >> parallel-restricted functions for each path generation function.
> >
> > Don't we anyway need that irrespective of caching it in RelOptInfo?
> > During bitmappath creation, bitmapqual could contain something
> > which needs to be evaluated for parallel-safety as it is built based
> > on index paths which inturn can be based on some join clause. As per
> > patch, the join clause parallel-safety is checked much later than
> > generation bitmappath.
>
> Yes, it's possible there could be some additional checks needed here
> for parameterized paths. But we're not quite there yet, so I think we
> can solve that problem when we get there. I have it in mind that in
> the future we may want a parallel_safe flag on each path, which would
> normally match the consider_parallel flag on the RelOptInfo but could
> instead be false if the path internally uses parallelism (since,
> currently, Gather nodes cannot be nested) or if it's got
> parallel-restricted parameterized quals. However, that seems like
> future work.
>
On Thu, Oct 22, 2015 at 11:59:58PM -0400, Robert Haas wrote: > On Thu, Oct 15, 2015 at 8:23 PM, Noah Misch <noah@leadboat.com> wrote: > > Agreed. More specifically, I had in mind for copyParamList() to check the > > mask while e.g. ExecEvalParamExtern() would either check nothing or merely > > assert that any mask included the requested parameter. It would be tricky to > > verify that as safe, so ... > > > >> Would it work to define this as "if non-NULL, > >> params lacking a 1-bit may be safely ignored"? Or some other tweak > >> that basically says that you don't need to care about this, but you > >> can if you want to. > > > > ... this is a better specification. > > Here's an attempt to implement that. Since that specification permits ParamListInfo consumers to ignore paramMask, the plpgsql_param_fetch() change from copy-paramlistinfo-fixes.patch is still formally required. > @@ -50,6 +51,7 @@ copyParamList(ParamListInfo from) > retval->parserSetup = NULL; > retval->parserSetupArg = NULL; > retval->numParams = from->numParams; > + retval->paramMask = bms_copy(from->paramMask); Considering that this function squashes the masked params, I wonder if it should just store NULL here. > > for (i = 0; i < from->numParams; i++) > { > @@ -58,6 +60,20 @@ copyParamList(ParamListInfo from) > int16 typLen; > bool typByVal; > > + /* > + * Ignore parameters we don't need, to save cycles and space, and > + * in case the fetch hook might fail. > + */ > + if (retval->paramMask != NULL && > + !bms_is_member(i, retval->paramMask)) The "and in case the fetch hook might fail" in this comment and its clones is contrary to the above specification. Under that specification, it would be a bug in the ParamListInfo producer to rely on consumers checking paramMask. Saving cycles/space would be the spec-approved paramMask use. Consider adding an XXX comment to the effect that cursors ought to stop using unshared param lists. The leading comment at setup_unshared_param_list() is a good home for such an addition.
On Fri, Oct 23, 2015 at 9:38 PM, Noah Misch <noah@leadboat.com> wrote: > Since that specification permits ParamListInfo consumers to ignore paramMask, > the plpgsql_param_fetch() change from copy-paramlistinfo-fixes.patch is still > formally required. So why am I not just doing that, then? Seems a lot more surgical. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Sat, Oct 24, 2015 at 07:49:07AM -0400, Robert Haas wrote: > On Fri, Oct 23, 2015 at 9:38 PM, Noah Misch <noah@leadboat.com> wrote: > > Since that specification permits ParamListInfo consumers to ignore paramMask, > > the plpgsql_param_fetch() change from copy-paramlistinfo-fixes.patch is still > > formally required. > > So why am I not just doing that, then? Seems a lot more surgical. do $$ declareparam_unused text := repeat('a', 100 * 1024 * 1024);param_used oid := 403; beginperform count(*) from pg_am where oid = param_used; end $$; I expect that if you were to inspect the EstimateParamListSpace() return values when executing that, you would find that it serializes the irrelevant 100 MiB datum. No possible logic in plpgsql_param_fetch() could stop that from happening, because copyParamList() and SerializeParamList() call the paramFetch hook only for dynamic parameters. Cursors faced the same problem, which is the raison d'être for setup_unshared_param_list().
On Sat, Oct 24, 2015 at 6:31 PM, Noah Misch <noah@leadboat.com> wrote: > On Sat, Oct 24, 2015 at 07:49:07AM -0400, Robert Haas wrote: >> On Fri, Oct 23, 2015 at 9:38 PM, Noah Misch <noah@leadboat.com> wrote: >> > Since that specification permits ParamListInfo consumers to ignore paramMask, >> > the plpgsql_param_fetch() change from copy-paramlistinfo-fixes.patch is still >> > formally required. >> >> So why am I not just doing that, then? Seems a lot more surgical. > > do $$ > declare > param_unused text := repeat('a', 100 * 1024 * 1024); > param_used oid := 403; > begin > perform count(*) from pg_am where oid = param_used; > end > $$; > > I expect that if you were to inspect the EstimateParamListSpace() return > values when executing that, you would find that it serializes the irrelevant > 100 MiB datum. No possible logic in plpgsql_param_fetch() could stop that > from happening, because copyParamList() and SerializeParamList() call the > paramFetch hook only for dynamic parameters. Cursors faced the same problem, > which is the raison d'être for setup_unshared_param_list(). Well, OK. That's not strictly a correctness issue, but here's an updated patch along the lines you suggested. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
On Wed, Oct 28, 2015 at 01:04:12AM +0100, Robert Haas wrote: > Well, OK. That's not strictly a correctness issue, but here's an > updated patch along the lines you suggested. > Finally, have setup_param_list set a new ParamListInfo field, > paramMask, to the parameters actually used in the expression, so that > we don't try to fetch those that are not needed when serializing a > parameter list. This isn't necessary for performance, but it makes s/performance/correctness/ > the performance of the parallel executor code comparable to what we > do for cases involving cursors. With that, the patch is ready.
On Fri, Oct 30, 2015 at 11:12 PM, Noah Misch <noah@leadboat.com> wrote: > On Wed, Oct 28, 2015 at 01:04:12AM +0100, Robert Haas wrote: >> Well, OK. That's not strictly a correctness issue, but here's an >> updated patch along the lines you suggested. > >> Finally, have setup_param_list set a new ParamListInfo field, >> paramMask, to the parameters actually used in the expression, so that >> we don't try to fetch those that are not needed when serializing a >> parameter list. This isn't necessary for performance, but it makes > > s/performance/correctness/ > >> the performance of the parallel executor code comparable to what we >> do for cases involving cursors. > > With that, the patch is ready. Thanks, committed. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Oct 23, 2015 at 10:33 AM, Robert Haas <robertmhaas@gmail.com> wrote:
Please find the rebased partial seq scan patch attached with this
mail.
Attachment
On Tue, Nov 3, 2015 at 9:41 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Oct 23, 2015 at 4:41 PM, Amit Kapila <amit.kapila16@gmail.com> > wrote: >> >> On Fri, Oct 23, 2015 at 10:33 AM, Robert Haas <robertmhaas@gmail.com> >> wrote: > > Please find the rebased partial seq scan patch attached with this > mail. > > Robert suggested me off list that we should once try to see if we > can use Seq Scan node instead of introducing a new Partial Seq Scan > node. I have analyzed to see if we can use the SeqScan node (containing > parallel flag) instead of introducing new partial seq scan and found that > we primarily need to change most of the functions in nodeSeqScan.c to > have a parallel flag check and do something special for Partial Seq Scan > and apart from that we need special handling in function > ExecSupportsBackwardScan(). In general, I think we can make > SeqScan node parallel-aware by having some special paths without > introducing much complexity and that can save us code-duplication > between nodeSeqScan.c and nodePartialSeqScan.c. One thing that makes > me slightly uncomfortable with this approach is that for partial seq scan, > currently the plan looks like: > > QUERY PLAN > -------------------------------------------------------------------------- > Gather (cost=0.00..2588194.25 rows=9990667 width=4) > Number of Workers: 1 > -> Partial Seq Scan on t1 (cost=0.00..89527.51 rows=9990667 width=4) > Filter: (c1 > 10000) > (4 rows) > > Now instead of displaying Partial Seq Scan, if we just display Seq Scan, > then it might confuse user, so it is better to add some thing indicating > parallel node if we want to go this route. IMO, the change from Partial Seq Scan to Seq Scan may not confuse user, if we clearly specify in the documentation that all plans under a Gather node are parallel plans. This is possible for the execution nodes that executes fully under a Gather node. The same is not possible for parallel aggregates, so we have to mention the aggregate node below Gather node as partial only. I feel this suggestion arises as may be because of some duplicate code between Partial Seq Scan and Seq scan. By using Seq Scan node only if we display as Partial Seq Scan by storing some flag in the plan? This avoids the need of adding new plan nodes. Regards, Hari Babu Fujitsu Australia
On Thu, Nov 5, 2015 at 12:52 AM, Haribabu Kommi <kommi.haribabu@gmail.com> wrote: >> Now instead of displaying Partial Seq Scan, if we just display Seq Scan, >> then it might confuse user, so it is better to add some thing indicating >> parallel node if we want to go this route. > > IMO, the change from Partial Seq Scan to Seq Scan may not confuse user, > if we clearly specify in the documentation that all plans under a Gather node > are parallel plans. > > This is possible for the execution nodes that executes fully under a > Gather node. > The same is not possible for parallel aggregates, so we have to mention the > aggregate node below Gather node as partial only. > > I feel this suggestion arises as may be because of some duplicate code between > Partial Seq Scan and Seq scan. By using Seq Scan node only if we display as > Partial Seq Scan by storing some flag in the plan? This avoids the > need of adding > new plan nodes. I was thinking about this idea: 1. Add a parallel_aware flag to each plan. 2. If the flag is set, have EXPLAIN print the word "Parallel" before the node name. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> I was thinking about this idea:
>
> 1. Add a parallel_aware flag to each plan.
>
On Thu, Nov 5, 2015 at 10:49 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Nov 5, 2015 at 11:54 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> I was thinking about this idea: >> >> 1. Add a parallel_aware flag to each plan. > > Okay, so shall we add it in generic Plan node or to specific plan nodes > like SeqScan, IndexScan, etc. To me, it appears that parallelism is > a node specific property, so we should add it to specific nodes and > for now as we are parallelising seq scan, so we can add this flag in > SeqScan node. What do you say? I think it should go in the Plan node itself. Parallel Append is going to need a way to test whether a node is parallel-aware, and there's nothing simpler than if (plan->parallel_aware). That makes life simple for EXPLAIN, too. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Fri, Oct 23, 2015 at 9:22 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > The base rel's consider_parallel flag won't be percolated to childrels, so > even > if we mark base rel as parallel capable, while generating the path it won't > be considered. I think we need to find a way to pass on that information if > we want to follow this way. Fixed in the attached version. I added a max_parallel_degree check, too, per your suggestion. > True, we can do that way. What I was trying to convey by above is > that we anyway need checks during path creation atleast in some > of the cases, so why not do all the checks at that time only as I > think all the information will be available at that time. > > I think if we store parallelism related info in RelOptInfo, that can also > be made to work, but the only worry I have with that approach is we > need to have checks at two levels one at RelOptInfo formation time > and other at Path formation time. I don't really see that as a problem. What I'm thinking about doing (but it's not implemented in the attached patch) is additionally adding a ppi_consider_parallel flag to the ParamPathInfo. This would be meaningful only for baserels, and would indicate whether the ParamPathInfo's ppi_clauses are parallel-safe. If we're thinking about adding a parallel path to a baserel, we need the RelOptInfo to have consider_parallel set and, if there is a ParamPathInfo, we need the ParamPathInfo's ppi_consider_parallel flag to be set also. That shows that both the rel's baserestrictinfo and the paramaterized join clauses are parallel-safe. For a joinrel, we can add a path if (1) the joinrel has consider_parallel set and (2) the paths being joined are parallel-safe. Testing condition (2) will require a per-Path flag, so we'll end up with one flag in the RelOptInfo, a second in the ParamPathInfo, and a third in the Path. That doesn't seem like a problem, though: it's a sign that we're doing this in a way that fits into the existing infrastructure, and it should be pretty efficient. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
>
> On Fri, Oct 23, 2015 at 9:22 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > The base rel's consider_parallel flag won't be percolated to childrels, so
> > even
> > if we mark base rel as parallel capable, while generating the path it won't
> > be considered. I think we need to find a way to pass on that information if
> > we want to follow this way.
>
> Fixed in the attached version. I added a max_parallel_degree check,
> too, per your suggestion.
>
> > True, we can do that way. What I was trying to convey by above is
> > that we anyway need checks during path creation atleast in some
> > of the cases, so why not do all the checks at that time only as I
> > think all the information will be available at that time.
> >
> > I think if we store parallelism related info in RelOptInfo, that can also
> > be made to work, but the only worry I have with that approach is we
> > need to have checks at two levels one at RelOptInfo formation time
> > and other at Path formation time.
>
> I don't really see that as a problem. What I'm thinking about doing
> (but it's not implemented in the attached patch) is additionally
> adding a ppi_consider_parallel flag to the ParamPathInfo. This would
> be meaningful only for baserels, and would indicate whether the
> ParamPathInfo's ppi_clauses are parallel-safe.
>
> If we're thinking about adding a parallel path to a baserel, we need
> the RelOptInfo to have consider_parallel set and, if there is a
> ParamPathInfo, we need the ParamPathInfo's ppi_consider_parallel flag
> to be set also. That shows that both the rel's baserestrictinfo and
> the paramaterized join clauses are parallel-safe. For a joinrel, we
> can add a path if (1) the joinrel has consider_parallel set and (2)
> the paths being joined are parallel-safe. Testing condition (2) will
> require a per-Path flag, so we'll end up with one flag in the
> RelOptInfo, a second in the ParamPathInfo, and a third in the Path.
>
> On Thu, Nov 5, 2015 at 10:49 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Thu, Nov 5, 2015 at 11:54 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >> I was thinking about this idea:
> >>
> >> 1. Add a parallel_aware flag to each plan.
> >
> > Okay, so shall we add it in generic Plan node or to specific plan nodes
> > like SeqScan, IndexScan, etc. To me, it appears that parallelism is
> > a node specific property, so we should add it to specific nodes and
> > for now as we are parallelising seq scan, so we can add this flag in
> > SeqScan node. What do you say?
>
> I think it should go in the Plan node itself. Parallel Append is
> going to need a way to test whether a node is parallel-aware, and
> there's nothing simpler than if (plan->parallel_aware). That makes
> life simple for EXPLAIN, too.
>
Attachment
On Mon, Nov 9, 2015 at 11:15 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Okay, I have updated the patch to make seq scan node parallel aware. > To make that happen we need to have parallel_aware flag both in Plan > as well as Path, so that we can pass that information from Path to Plan. > I think the right place to copy parallel_aware info from path to > plan is copy_path_costsize and ideally we should change the name > of function to something like copy_generic_path_info(), but for > now I have retained it's original name as it is used at many places, > let me know if you think we should goahead and change the name > of function as well. > > I have changed Explain as well so that it adds Parallel for Seq Scan if > SeqScan node is parallel_aware. > > I have not integrated it with consider-parallel patch, so that this and > Partial Seq Scan version of the patch can be compared without much > difficulity. > > Thoughts? I've committed most of this, except for some planner bits that I didn't like, and after a bunch of cleanup. Instead, I committed the consider-parallel-v2.patch with some additional planner bits to make up for the ones I removed from your patch. So, now we have parallel sequential scan! For those following along at home, here's a demo: rhaas=# \timing Timing is on. rhaas=# select * from pgbench_accounts where filler like '%a%';aid | bid | abalance | filler -----+-----+----------+-------- (0 rows) Time: 743.061 ms rhaas=# set max_parallel_degree = 4; SET Time: 0.270 ms rhaas=# select * from pgbench_accounts where filler like '%a%';aid | bid | abalance | filler -----+-----+----------+-------- (0 rows) Time: 213.412 ms This is all pretty primitive at this point - there are still lots of things that need to be fixed and improved, and it applies to only the very simplest cases at present, but, hey, parallel query. Check it out. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Nov 11, 2015 at 11:53 PM, Robert Haas <robertmhaas@gmail.com> wrote: > For those following along at home, here's a demo: > > rhaas=# \timing > Timing is on. > rhaas=# select * from pgbench_accounts where filler like '%a%'; > aid | bid | abalance | filler > -----+-----+----------+-------- > (0 rows) > > Time: 743.061 ms > rhaas=# set max_parallel_degree = 4; > SET > Time: 0.270 ms > rhaas=# select * from pgbench_accounts where filler like '%a%'; > aid | bid | abalance | filler > -----+-----+----------+-------- > (0 rows) > > Time: 213.412 ms > > This is all pretty primitive at this point - there are still lots of > things that need to be fixed and improved, and it applies to only the > very simplest cases at present, but, hey, parallel query. Check it > out. Yay! Great work guys! Thanks, Amit
On 11 November 2015 at 14:53, Robert Haas <robertmhaas@gmail.com> wrote: > On Mon, Nov 9, 2015 at 11:15 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> Okay, I have updated the patch to make seq scan node parallel aware. >> To make that happen we need to have parallel_aware flag both in Plan >> as well as Path, so that we can pass that information from Path to Plan. >> I think the right place to copy parallel_aware info from path to >> plan is copy_path_costsize and ideally we should change the name >> of function to something like copy_generic_path_info(), but for >> now I have retained it's original name as it is used at many places, >> let me know if you think we should goahead and change the name >> of function as well. >> >> I have changed Explain as well so that it adds Parallel for Seq Scan if >> SeqScan node is parallel_aware. >> >> I have not integrated it with consider-parallel patch, so that this and >> Partial Seq Scan version of the patch can be compared without much >> difficulity. >> >> Thoughts? > > I've committed most of this, except for some planner bits that I > didn't like, and after a bunch of cleanup. Instead, I committed the > consider-parallel-v2.patch with some additional planner bits to make > up for the ones I removed from your patch. So, now we have parallel > sequential scan! > > For those following along at home, here's a demo: > > rhaas=# \timing > Timing is on. > rhaas=# select * from pgbench_accounts where filler like '%a%'; > aid | bid | abalance | filler > -----+-----+----------+-------- > (0 rows) > > Time: 743.061 ms > rhaas=# set max_parallel_degree = 4; > SET > Time: 0.270 ms > rhaas=# select * from pgbench_accounts where filler like '%a%'; > aid | bid | abalance | filler > -----+-----+----------+-------- > (0 rows) > > Time: 213.412 ms > > This is all pretty primitive at this point - there are still lots of > things that need to be fixed and improved, and it applies to only the > very simplest cases at present, but, hey, parallel query. Check it > out. Congratulations to both you and Amit. This is a significant landmark in PostgreSQL feature development. Thom
Congratulations to both you and Amit. This is a significant landmarkOn 11 November 2015 at 14:53, Robert Haas <robertmhaas@gmail.com> wrote:
> On Mon, Nov 9, 2015 at 11:15 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
>> Okay, I have updated the patch to make seq scan node parallel aware.
>> To make that happen we need to have parallel_aware flag both in Plan
>> as well as Path, so that we can pass that information from Path to Plan.
>> I think the right place to copy parallel_aware info from path to
>> plan is copy_path_costsize and ideally we should change the name
>> of function to something like copy_generic_path_info(), but for
>> now I have retained it's original name as it is used at many places,
>> let me know if you think we should goahead and change the name
>> of function as well.
>>
>> I have changed Explain as well so that it adds Parallel for Seq Scan if
>> SeqScan node is parallel_aware.
>>
>> I have not integrated it with consider-parallel patch, so that this and
>> Partial Seq Scan version of the patch can be compared without much
>> difficulity.
>>
>> Thoughts?
>
> I've committed most of this, except for some planner bits that I
> didn't like, and after a bunch of cleanup. Instead, I committed the
> consider-parallel-v2.patch with some additional planner bits to make
> up for the ones I removed from your patch. So, now we have parallel
> sequential scan!
>
> For those following along at home, here's a demo:
>
> rhaas=# \timing
> Timing is on.
> rhaas=# select * from pgbench_accounts where filler like '%a%';
> aid | bid | abalance | filler
> -----+-----+----------+--------
> (0 rows)
>
> Time: 743.061 ms
> rhaas=# set max_parallel_degree = 4;
> SET
> Time: 0.270 ms
> rhaas=# select * from pgbench_accounts where filler like '%a%';
> aid | bid | abalance | filler
> -----+-----+----------+--------
> (0 rows)
>
> Time: 213.412 ms
>
> This is all pretty primitive at this point - there are still lots of
> things that need to be fixed and improved, and it applies to only the
> very simplest cases at present, but, hey, parallel query. Check it
> out.
in PostgreSQL feature development.
+1
Thom
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
postgres=# set max_parallel_degree to 4;
SET
Time: 0.717 ms
postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
┌───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
│ QUERY PLAN │
╞═══════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡
│ Aggregate (cost=9282.50..9282.51 rows=1 width=0) (actual time=142.541..142.541 rows=1 loops=1) │
│ -> Gather (cost=1000.00..9270.00 rows=5000 width=0) (actual time=0.633..130.926 rows=100000 loops=1) │
│ Number of Workers: 2 │
│ -> Parallel Seq Scan on xxx (cost=0.00..7770.00 rows=5000 width=0) (actual time=0.052..411.303 rows=169631 loops=1) │
│ Filter: ((a % 10) = 0) │
│ Rows Removed by Filter: 1526399 │
│ Planning time: 0.167 ms │
│ Execution time: 144.519 ms │
└───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
(8 rows)
Time: 145.374 ms
postgres=# set max_parallel_degree to 1;
SET
Time: 0.706 ms
postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
┌────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
│ QUERY PLAN │
╞════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡
│ Aggregate (cost=14462.50..14462.51 rows=1 width=0) (actual time=163.355..163.355 rows=1 loops=1) │
│ -> Gather (cost=1000.00..14450.00 rows=5000 width=0) (actual time=0.485..152.827 rows=100000 loops=1) │
│ Number of Workers: 1 │
│ -> Parallel Seq Scan on xxx (cost=0.00..12950.00 rows=5000 width=0) (actual time=0.043..309.740 rows=145364 loops=1) │
│ Filter: ((a % 10) = 0) │
│ Rows Removed by Filter: 1308394 │
│ Planning time: 0.129 ms │
│ Execution time: 165.102 ms │
└────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
(8 rows)
On 11 November 2015 at 17:59, Pavel Stehule <pavel.stehule@gmail.com> wrote: > Hi > > I have a first query > > I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are > differen > > postgres=# set max_parallel_degree to 4; > SET > Time: 0.717 ms > postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0; > ┌───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ > │ QUERY PLAN > │ > ╞═══════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡ > │ Aggregate (cost=9282.50..9282.51 rows=1 width=0) (actual > time=142.541..142.541 rows=1 loops=1) │ > │ -> Gather (cost=1000.00..9270.00 rows=5000 width=0) (actual > time=0.633..130.926 rows=100000 loops=1) │ > │ Number of Workers: 2 > │ > │ -> Parallel Seq Scan on xxx (cost=0.00..7770.00 rows=5000 > width=0) (actual time=0.052..411.303 rows=169631 loops=1) │ > │ Filter: ((a % 10) = 0) > │ > │ Rows Removed by Filter: 1526399 > │ > │ Planning time: 0.167 ms > │ > │ Execution time: 144.519 ms > │ > └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ > (8 rows) > > Time: 145.374 ms > postgres=# set max_parallel_degree to 1; > SET > Time: 0.706 ms > postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0; > ┌────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐ > │ QUERY PLAN > │ > ╞════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡ > │ Aggregate (cost=14462.50..14462.51 rows=1 width=0) (actual > time=163.355..163.355 rows=1 loops=1) │ > │ -> Gather (cost=1000.00..14450.00 rows=5000 width=0) (actual > time=0.485..152.827 rows=100000 loops=1) │ > │ Number of Workers: 1 > │ > │ -> Parallel Seq Scan on xxx (cost=0.00..12950.00 rows=5000 > width=0) (actual time=0.043..309.740 rows=145364 loops=1) │ > │ Filter: ((a % 10) = 0) > │ > │ Rows Removed by Filter: 1308394 > │ > │ Planning time: 0.129 ms > │ > │ Execution time: 165.102 ms > │ > └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘ > (8 rows) > > Rows removed by filter: 1308394 X 1526399. Is it expected? Yeah, I noticed the same thing, but more pronounced: With set max_parallel_degree = 4: # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->0->>'term' like 'design%' or content->'tags'->0->>'term' like 'web%'; QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=49575.51..49575.52 rows=1 width=0) (actual time=744.267..744.267 rows=1 loops=1) Output: count(*) Buffers: shared hit=175423 -> Gather (cost=1000.00..49544.27rows=12496 width=0) (actual time=0.351..731.662 rows=55151 loops=1) Output: content Number of Workers: 4 Buffers: shared hit=175423 -> Parallel Seq Scan on public.js (cost=0.00..47294.67 rows=12496 width=0) (actual time=0.030..5912.118 rows=96062 loops=1) Output: content Filter: (((((js.content-> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'web%'::text)) Rows Removed by Filter: 2085546 Buffers: shared hit=305123Planningtime: 0.123 msExecution time: 759.313 ms (14 rows) With set max_parallel_degree = 0: # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->0->>'term' like 'design%' or content->'tags'->0->>'term' like 'web%'; QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=212857.25..212857.26 rows=1 width=0) (actual time=1235.082..1235.082 rows=1 loops=1) Output: count(*) Buffers: shared hit=175243 -> Seq Scan on public.js (cost=0.00..212826.01rows=12496 width=0) (actual time=0.019..1228.515 rows=55151 loops=1) Output: content Filter: (((((js.content -> 'tags'::text)-> 0) ->> 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'web%'::text)) Rows Removed by Filter: 1197822 Buffers: shared hit=175243Planningtime: 0.064 msExecution time: 1235.108 ms (10 rows) Time: 1235.517 ms Rows removed: 2085546 vs 1197822 Buffers hit: 305123 vs 175243 Thom
Yeah, I noticed the same thing, but more pronounced:On 11 November 2015 at 17:59, Pavel Stehule <pavel.stehule@gmail.com> wrote:
> Hi
>
> I have a first query
>
> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are
> differen
>
> postgres=# set max_parallel_degree to 4;
> SET
> Time: 0.717 ms
> postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
> ┌───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
> │ QUERY PLAN
> │
> ╞═══════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡
> │ Aggregate (cost=9282.50..9282.51 rows=1 width=0) (actual
> time=142.541..142.541 rows=1 loops=1) │
> │ -> Gather (cost=1000.00..9270.00 rows=5000 width=0) (actual
> time=0.633..130.926 rows=100000 loops=1) │
> │ Number of Workers: 2
> │
> │ -> Parallel Seq Scan on xxx (cost=0.00..7770.00 rows=5000
> width=0) (actual time=0.052..411.303 rows=169631 loops=1) │
> │ Filter: ((a % 10) = 0)
> │
> │ Rows Removed by Filter: 1526399
> │
> │ Planning time: 0.167 ms
> │
> │ Execution time: 144.519 ms
> │
> └───────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
> (8 rows)
>
> Time: 145.374 ms
> postgres=# set max_parallel_degree to 1;
> SET
> Time: 0.706 ms
> postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
> ┌────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┐
> │ QUERY PLAN
> │
> ╞════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════╡
> │ Aggregate (cost=14462.50..14462.51 rows=1 width=0) (actual
> time=163.355..163.355 rows=1 loops=1) │
> │ -> Gather (cost=1000.00..14450.00 rows=5000 width=0) (actual
> time=0.485..152.827 rows=100000 loops=1) │
> │ Number of Workers: 1
> │
> │ -> Parallel Seq Scan on xxx (cost=0.00..12950.00 rows=5000
> width=0) (actual time=0.043..309.740 rows=145364 loops=1) │
> │ Filter: ((a % 10) = 0)
> │
> │ Rows Removed by Filter: 1308394
> │
> │ Planning time: 0.129 ms
> │
> │ Execution time: 165.102 ms
> │
> └────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────────┘
> (8 rows)
>
> Rows removed by filter: 1308394 X 1526399. Is it expected?
With set max_parallel_degree = 4:
# explain (analyse, buffers, timing, verbose, costs) select count(*)
from js where content->'tags'->0->>'term' like 'design%' or
content->'tags'->0->>'term' like 'web%';
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=49575.51..49575.52 rows=1 width=0) (actual
time=744.267..744.267 rows=1 loops=1)
Output: count(*)
Buffers: shared hit=175423
-> Gather (cost=1000.00..49544.27 rows=12496 width=0) (actual
time=0.351..731.662 rows=55151 loops=1)
Output: content
Number of Workers: 4
Buffers: shared hit=175423
-> Parallel Seq Scan on public.js (cost=0.00..47294.67
rows=12496 width=0) (actual time=0.030..5912.118 rows=96062 loops=1)
Output: content
Filter: (((((js.content -> 'tags'::text) -> 0) ->>
'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text)
-> 0) ->> 'term'::text) ~~ 'web%'::text))
Rows Removed by Filter: 2085546
Buffers: shared hit=305123
Planning time: 0.123 ms
Execution time: 759.313 ms
(14 rows)
With set max_parallel_degree = 0:
# explain (analyse, buffers, timing, verbose, costs) select count(*)
from js where content->'tags'->0->>'term' like 'design%' or
content->'tags'->0->>'term' like 'web%';
QUERY PLAN
---------------------------------------------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=212857.25..212857.26 rows=1 width=0) (actual
time=1235.082..1235.082 rows=1 loops=1)
Output: count(*)
Buffers: shared hit=175243
-> Seq Scan on public.js (cost=0.00..212826.01 rows=12496
width=0) (actual time=0.019..1228.515 rows=55151 loops=1)
Output: content
Filter: (((((js.content -> 'tags'::text) -> 0) ->>
'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text)
-> 0) ->> 'term'::text) ~~ 'web%'::text))
Rows Removed by Filter: 1197822
Buffers: shared hit=175243
Planning time: 0.064 ms
Execution time: 1235.108 ms
(10 rows)
Time: 1235.517 ms
Rows removed: 2085546 vs 1197822
Buffers hit: 305123 vs 175243
Thom
On Wed, Nov 11, 2015 at 12:59 PM, Pavel Stehule <pavel.stehule@gmail.com> wrote: > I have a first query > > I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are > differen Hmm, I see I was right about people finding more bugs once this was committed. That didn't take long. There's supposed to be code to handle this - see the SharedPlanStateInstrumentation stuff in execParallel.c - but it's evidently a few bricks shy of a load. ExecParallelReportInstrumentation is supposed to transfer the counts from each worker to the DSM: ps_instrument = &instrumentation->ps_instrument[i]; SpinLockAcquire(&ps_instrument->mutex); InstrAggNode(&ps_instrument->instr,planstate->instrument); SpinLockRelease(&ps_instrument->mutex); And ExecParallelRetrieveInstrumentation is supposed to slurp those counts back into the leader's PlanState objects: /* No need to acquire the spinlock here; workers have exited already. */ ps_instrument = &instrumentation->ps_instrument[i]; InstrAggNode(planstate->instrument, &ps_instrument->instr); This might be a race condition, or it might be just wrong logic. Could you test what happens if you insert something like a 1-second sleep in ExecParallelFinish just after the call to WaitForParallelWorkersToFinish()? If that makes the results consistent, this is a race. If it doesn't, something else is wrong: then it would be useful to know whether the workers are actually calling ExecParallelReportInstrumentation, and whether the leader is actually calling ExecParallelRetrieveInstrumentation, and if so whether they are doing it for the correct set of nodes. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Wed, Nov 11, 2015 at 12:59 PM, Pavel Stehule <pavel.stehule@gmail.com> wrote:
> I have a first query
>
> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are
> differen
Hmm, I see I was right about people finding more bugs once this was
committed. That didn't take long.
There's supposed to be code to handle this - see the
SharedPlanStateInstrumentation stuff in execParallel.c - but it's
evidently a few bricks shy of a load.
ExecParallelReportInstrumentation is supposed to transfer the counts
from each worker to the DSM:
ps_instrument = &instrumentation->ps_instrument[i];
SpinLockAcquire(&ps_instrument->mutex);
InstrAggNode(&ps_instrument->instr, planstate->instrument);
SpinLockRelease(&ps_instrument->mutex);
And ExecParallelRetrieveInstrumentation is supposed to slurp those
counts back into the leader's PlanState objects:
/* No need to acquire the spinlock here; workers have exited already. */
ps_instrument = &instrumentation->ps_instrument[i];
InstrAggNode(planstate->instrument, &ps_instrument->instr);
This might be a race condition, or it might be just wrong logic.
Could you test what happens if you insert something like a 1-second
sleep in ExecParallelFinish just after the call to
WaitForParallelWorkersToFinish()? If that makes the results
consistent, this is a race. If it doesn't, something else is wrong:
then it would be useful to know whether the workers are actually
calling ExecParallelReportInstrumentation, and whether the leader is
actually calling ExecParallelRetrieveInstrumentation, and if so
whether they are doing it for the correct set of nodes.
postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
QUERY PLAN
═════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════════
Aggregate (cost=9282.50..9282.51 rows=1 width=0) (actual time=154.535..154.535 rows=1 loops=1)
-> Gather (cost=1000.00..9270.00 rows=5000 width=0) (actual time=0.675..142.320 rows=100000 loops=1)
Number of Workers: 2
-> Parallel Seq Scan on xxx (cost=0.00..7770.00 rows=5000 width=0) (actual time=0.075..445.999 rows=168927 loops=1)
Filter: ((a % 10) = 0)
Rows Removed by Filter: 1520549
Planning time: 0.117 ms
Execution time: 1155.505 ms
(8 rows)
postgres=# EXPLAIN ANALYZE select count(*) from xxx where a % 10 = 0;
QUERY PLAN
═══════════════════════════════════════════════════════════════════════════════════════════════════════════════
Aggregate (cost=19437.50..19437.51 rows=1 width=0) (actual time=171.233..171.233 rows=1 loops=1)
-> Seq Scan on xxx (cost=0.00..19425.00 rows=5000 width=0) (actual time=0.187..162.627 rows=100000 loops=1)
Filter: ((a % 10) = 0)
Rows Removed by Filter: 900000
Planning time: 0.119 ms
Execution time: 171.322 ms
(6 rows)
create table xxx(a int);
insert into xxx select generate_series(1,1000000);
On 11 November 2015 at 19:26, Robert Haas <robertmhaas@gmail.com> wrote: > On Wed, Nov 11, 2015 at 12:59 PM, Pavel Stehule <pavel.stehule@gmail.com> wrote: >> I have a first query >> >> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are >> differen > > Hmm, I see I was right about people finding more bugs once this was > committed. That didn't take long. > > There's supposed to be code to handle this - see the > SharedPlanStateInstrumentation stuff in execParallel.c - but it's > evidently a few bricks shy of a load. > ExecParallelReportInstrumentation is supposed to transfer the counts > from each worker to the DSM: > > ps_instrument = &instrumentation->ps_instrument[i]; > SpinLockAcquire(&ps_instrument->mutex); > InstrAggNode(&ps_instrument->instr, planstate->instrument); > SpinLockRelease(&ps_instrument->mutex); > > And ExecParallelRetrieveInstrumentation is supposed to slurp those > counts back into the leader's PlanState objects: > > /* No need to acquire the spinlock here; workers have exited already. */ > ps_instrument = &instrumentation->ps_instrument[i]; > InstrAggNode(planstate->instrument, &ps_instrument->instr); > > This might be a race condition, or it might be just wrong logic. > Could you test what happens if you insert something like a 1-second > sleep in ExecParallelFinish just after the call to > WaitForParallelWorkersToFinish()? If that makes the results > consistent, this is a race. If it doesn't, something else is wrong: > then it would be useful to know whether the workers are actually > calling ExecParallelReportInstrumentation, and whether the leader is > actually calling ExecParallelRetrieveInstrumentation, and if so > whether they are doing it for the correct set of nodes. Hmm.. I made the change, but clearly it's not sleeping properly with my change (I'm expecting a total runtime in excess of 1 second): max_parallel_degree = 4: # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->0->>'term' like 'design%' or content->'tags'->0->>'term' like 'web%'; QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=49578.18..49578.19 rows=1 width=0) (actual time=797.518..797.518 rows=1 loops=1) Output: count(*) Buffers: shared hit=174883 read=540 -> Gather (cost=1000.00..49546.93rows=12500 width=0) (actual time=0.245..784.959 rows=55151 loops=1) Output: content Number of Workers: 4 Buffers: shared hit=174883read=540 -> Parallel Seq Scan on public.js (cost=0.00..47296.93 rows=12500 width=0) (actual time=0.019..6153.679 rows=94503 loops=1) Output: content Filter: (((((js.content-> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'web%'::text)) Rows Removed by Filter: 2051330 Buffers: shared hit=299224read=907Planning time: 0.086 msExecution time: 803.026 ms max_parallel_degree = 0: # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->0->>'term' like 'design%' or content->'tags'->0->>'term' like 'web%'; QUERY PLAN ---------------------------------------------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=212867.43..212867.44 rows=1 width=0) (actual time=1278.717..1278.717 rows=1 loops=1) Output: count(*) Buffers: shared hit=174671 read=572 -> Seq Scan on public.js (cost=0.00..212836.18 rows=12500 width=0) (actual time=0.018..1272.030 rows=55151 loops=1) Output: content Filter: (((((js.content -> 'tags'::text)-> 0) ->> 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) -> 0) ->> 'term'::text) ~~ 'web%'::text)) Rows Removed by Filter: 1197822 Buffers: shared hit=174671 read=572Planningtime: 0.064 msExecution time: 1278.741 ms (10 rows) Time: 1279.145 ms I did, however, notice that repeated runs of the query with max_parallel_degree = 4 yields different counts of rows removed by filter: Run 1: 2051330 Run 2: 2081252 Run 3: 2065112 Run 4: 2022045 Run 5: 2025384 Run 6: 2059360 Run 7: 2079620 Run 8: 2058541 -- Thom
On 11 November 2015 at 19:51, Thom Brown <thom@linux.com> wrote: > On 11 November 2015 at 19:26, Robert Haas <robertmhaas@gmail.com> wrote: >> On Wed, Nov 11, 2015 at 12:59 PM, Pavel Stehule <pavel.stehule@gmail.com> wrote: >>> I have a first query >>> >>> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are >>> differen >> >> Hmm, I see I was right about people finding more bugs once this was >> committed. That didn't take long. >> >> There's supposed to be code to handle this - see the >> SharedPlanStateInstrumentation stuff in execParallel.c - but it's >> evidently a few bricks shy of a load. >> ExecParallelReportInstrumentation is supposed to transfer the counts >> from each worker to the DSM: >> >> ps_instrument = &instrumentation->ps_instrument[i]; >> SpinLockAcquire(&ps_instrument->mutex); >> InstrAggNode(&ps_instrument->instr, planstate->instrument); >> SpinLockRelease(&ps_instrument->mutex); >> >> And ExecParallelRetrieveInstrumentation is supposed to slurp those >> counts back into the leader's PlanState objects: >> >> /* No need to acquire the spinlock here; workers have exited already. */ >> ps_instrument = &instrumentation->ps_instrument[i]; >> InstrAggNode(planstate->instrument, &ps_instrument->instr); >> >> This might be a race condition, or it might be just wrong logic. >> Could you test what happens if you insert something like a 1-second >> sleep in ExecParallelFinish just after the call to >> WaitForParallelWorkersToFinish()? If that makes the results >> consistent, this is a race. If it doesn't, something else is wrong: >> then it would be useful to know whether the workers are actually >> calling ExecParallelReportInstrumentation, and whether the leader is >> actually calling ExecParallelRetrieveInstrumentation, and if so >> whether they are doing it for the correct set of nodes. > > Hmm.. I made the change, but clearly it's not sleeping properly with > my change (I'm expecting a total runtime in excess of 1 second): > > max_parallel_degree = 4: > > # explain (analyse, buffers, timing, verbose, costs) select count(*) > from js where content->'tags'->0->>'term' like 'design%' or > content->'tags'->0->>'term' like 'web%'; > > QUERY PLAN > --------------------------------------------------------------------------------------------------------------------------------------------------------------------------------- > Aggregate (cost=49578.18..49578.19 rows=1 width=0) (actual > time=797.518..797.518 rows=1 loops=1) > Output: count(*) > Buffers: shared hit=174883 read=540 > -> Gather (cost=1000.00..49546.93 rows=12500 width=0) (actual > time=0.245..784.959 rows=55151 loops=1) > Output: content > Number of Workers: 4 > Buffers: shared hit=174883 read=540 > -> Parallel Seq Scan on public.js (cost=0.00..47296.93 > rows=12500 width=0) (actual time=0.019..6153.679 rows=94503 loops=1) > Output: content > Filter: (((((js.content -> 'tags'::text) -> 0) ->> > 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) > -> 0) ->> 'term'::text) ~~ 'web%'::text)) > Rows Removed by Filter: 2051330 > Buffers: shared hit=299224 read=907 > Planning time: 0.086 ms > Execution time: 803.026 ms > > > max_parallel_degree = 0: > > # explain (analyse, buffers, timing, verbose, costs) select count(*) > from js where content->'tags'->0->>'term' like 'design%' or > content->'tags'->0->>'term' like 'web%'; > > QUERY PLAN > --------------------------------------------------------------------------------------------------------------------------------------------------------------------------- > Aggregate (cost=212867.43..212867.44 rows=1 width=0) (actual > time=1278.717..1278.717 rows=1 loops=1) > Output: count(*) > Buffers: shared hit=174671 read=572 > -> Seq Scan on public.js (cost=0.00..212836.18 rows=12500 > width=0) (actual time=0.018..1272.030 rows=55151 loops=1) > Output: content > Filter: (((((js.content -> 'tags'::text) -> 0) ->> > 'term'::text) ~~ 'design%'::text) OR ((((js.content -> 'tags'::text) > -> 0) ->> 'term'::text) ~~ 'web%'::text)) > Rows Removed by Filter: 1197822 > Buffers: shared hit=174671 read=572 > Planning time: 0.064 ms > Execution time: 1278.741 ms > (10 rows) > > Time: 1279.145 ms > > > I did, however, notice that repeated runs of the query with > max_parallel_degree = 4 yields different counts of rows removed by > filter: > > Run 1: 2051330 > Run 2: 2081252 > Run 3: 2065112 > Run 4: 2022045 > Run 5: 2025384 > Run 6: 2059360 > Run 7: 2079620 > Run 8: 2058541 Here's another oddity, with max_parallel_degree = 1: # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->>'title' like '%design%'; QUERY PLAN ------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=132489.34..132489.35 rows=1 width=0) (actual time=382.987..382.987 rows=1 loops=1) Output: count(*) Buffers: shared hit=175288 -> Gather (cost=1000.00..132488.34rows=401 width=0) (actual time=382.983..382.983 rows=0 loops=1) Output: content Number of Workers: 1 Buffers: shared hit=175288 -> Parallel Seq Scan on public.js (cost=0.00..131448.24 rows=401 width=0) (actual time=379.407..1141.437 rows=0 loops=1) Output: content Filter: (((js.content-> 'tags'::text) ->> 'title'::text) ~~ '%design%'::text) Rows Removed by Filter: 1724810 Buffers: shared hit=241201Planningtime: 0.104 msExecution time: 403.045 ms (14 rows) Time: 403.596 ms The actual time of the sequential scan was 1141.437ms, but the total execution time was 403.045ms. And successive runs with max_parallel_degree = 1 also yield a different number of rows removed by the filter, as well as a different number of buffers being hit: Run: rows removed / buffers hit 1: 1738517 / 243143 2: 1729361 / 241900 3: 1737168 / 242974 4: 1734440 / 242591 Thom
On 2015/11/12 4:26, Robert Haas wrote: > On Wed, Nov 11, 2015 at 12:59 PM, Pavel Stehule <pavel.stehule@gmail.com> wrote: >> I have a first query >> >> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are >> differen > > Hmm, I see I was right about people finding more bugs once this was > committed. That didn't take long. I encountered one more odd behavior: postgres=# EXPLAIN ANALYZE SELECT abalance FROM pgbench_accounts WHERE aid = 23466; QUERY PLAN -----------------------------------------------------------------------------------------------------------------------------------Gather (cost=1000.00..65207.88 rows=1 width=4) (actual time=17450.595..17451.151 rows=1 loops=1) Number of Workers: 4 -> Parallel Seq Scan on pgbench_accounts (cost=0.00..64207.78rows=1 width=4) (actual time=55.934..157001.134 rows=2 loops=1) Filter: (aid = 23466) Rows Removed by Filter: 18047484Planningtime: 0.198 msExecution time: 17453.565 ms (7 rows) The #rows removed here is almost twice the number of rows in the table (10m). Also, the #rows selected shown is 2 for Parallel Seq Scan whereas only 1 row is selected. Thanks, Amit
>
> Hi
>
> I have a first query
>
> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are differen
>
Thanks for the report. The reason for this problem is that instrumentation
Attachment
On 12 November 2015 at 15:23, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Nov 11, 2015 at 11:29 PM, Pavel Stehule <pavel.stehule@gmail.com> > wrote: >> >> Hi >> >> I have a first query >> >> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are >> differen >> > > Thanks for the report. The reason for this problem is that instrumentation > information from workers is getting aggregated multiple times. In > ExecShutdownGatherWorkers(), we call ExecParallelFinish where it > will wait for workers to finish and then accumulate stats from workers. > Now ExecShutdownGatherWorkers() could be called multiple times > (once we read all tuples from workers, at end of node) and it should be > ensured that repeated calls should not try to redo the work done by first > call. > The same is ensured for tuplequeues, but not for parallel executor info. > I think we can safely assume that we need to call ExecParallelFinish() only > when there are workers started by the Gathers node, so on those lines > attached patch should fix the problem. That fixes the count issue for me, although not the number of buffers hit, or the actual time taken. Thom
>
> On 12 November 2015 at 15:23, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Wed, Nov 11, 2015 at 11:29 PM, Pavel Stehule <pavel.stehule@gmail.com>
> > wrote:
> >>
> >> Hi
> >>
> >> I have a first query
> >>
> >> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are
> >> differen
> >>
> >
> > Thanks for the report. The reason for this problem is that instrumentation
> > information from workers is getting aggregated multiple times. In
> > ExecShutdownGatherWorkers(), we call ExecParallelFinish where it
> > will wait for workers to finish and then accumulate stats from workers.
> > Now ExecShutdownGatherWorkers() could be called multiple times
> > (once we read all tuples from workers, at end of node) and it should be
> > ensured that repeated calls should not try to redo the work done by first
> > call.
> > The same is ensured for tuplequeues, but not for parallel executor info.
> > I think we can safely assume that we need to call ExecParallelFinish() only
> > when there are workers started by the Gathers node, so on those lines
> > attached patch should fix the problem.
>
> That fixes the count issue for me, although not the number of buffers
> hit,
QUERY
PLAN
------------------------------------------------------------------------------------------------------------------------------------
Aggregate (cost=132489.34..132489.35 rows=1 width=0) (actual
time=382.987..382.987 rows=1 loops=1)
Output: count(*)
Buffers: shared hit=175288
-> Gather (cost=1000.00..132488.34 rows=401 width=0) (actual
time=382.983..382.983 rows=0 loops=1)
Output: content
Number of Workers: 1
Buffers: shared hit=175288
-> Parallel Seq Scan on public.js (cost=0.00..131448.24
rows=401 width=0) (actual time=379.407..1141.437 rows=0 loops=1)
Output: content
Filter: (((js.content -> 'tags'::text) ->>
'title'::text) ~~ '%design%'::text)
Rows Removed by Filter: 1724810
Buffers: shared hit=241201
Planning time: 0.104 ms
Execution time: 403.045 ms
(14 rows)
Time: 403.596 ms
>
On Thu, Nov 12, 2015 at 10:39 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > The number of shared buffers hit could be different across different runs > because the read sequence of parallel workers can't be guaranteed, also > I don't think same is even guaranteed for Seq Scan node, The number of hits could be different. However, it seems like any sequential scan, parallel or not, should have a number of accesses (hit + read) equal to the size of the relation. Not sure if that's what is happening here. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 13 November 2015 at 03:39, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Thu, Nov 12, 2015 at 9:05 PM, Thom Brown <thom@linux.com> wrote: >> >> On 12 November 2015 at 15:23, Amit Kapila <amit.kapila16@gmail.com> wrote: >> > On Wed, Nov 11, 2015 at 11:29 PM, Pavel Stehule >> > <pavel.stehule@gmail.com> >> > wrote: >> >> >> >> Hi >> >> >> >> I have a first query >> >> >> >> I looked on EXPLAIN ANALYZE output and the numbers of filtered rows are >> >> differen >> >> >> > >> > Thanks for the report. The reason for this problem is that >> > instrumentation >> > information from workers is getting aggregated multiple times. In >> > ExecShutdownGatherWorkers(), we call ExecParallelFinish where it >> > will wait for workers to finish and then accumulate stats from workers. >> > Now ExecShutdownGatherWorkers() could be called multiple times >> > (once we read all tuples from workers, at end of node) and it should be >> > ensured that repeated calls should not try to redo the work done by >> > first >> > call. >> > The same is ensured for tuplequeues, but not for parallel executor info. >> > I think we can safely assume that we need to call ExecParallelFinish() >> > only >> > when there are workers started by the Gathers node, so on those lines >> > attached patch should fix the problem. >> >> That fixes the count issue for me, although not the number of buffers >> hit, >> > > The number of shared buffers hit could be different across different runs > because the read sequence of parallel workers can't be guaranteed, also > I don't think same is even guaranteed for Seq Scan node, the other > operations > in parallel could lead to different number, however the actual problem was > that in one of the plans shown by you [1], the Buffers hit at Gather node > (175288) is lesser than the Buffers hit at Parallel Seq Scan node (241201). > Do you still (after applying above patch) see that Gather node is showing > lesser hit buffers than Parallel Seq Scan node? Hmm... that's odd, I'm not seeing the problem now, so maybe I'm mistaken there. > [1] > # explain (analyse, buffers, timing, verbose, costs) select count(*) > from js where content->'tags'->>'title' like '%design%'; > QUERY > PLAN > ------------------------------------------------------------------------------------------------------------------------------------ > Aggregate (cost=132489.34..132489.35 rows=1 width=0) (actual > time=382.987..382.987 rows=1 loops=1) > Output: count(*) > Buffers: shared hit=175288 > -> Gather (cost=1000.00..132488.34 rows=401 width=0) (actual > time=382.983..382.983 rows=0 loops=1) > Output: content > Number of Workers: 1 > Buffers: shared hit=175288 > -> Parallel Seq Scan on public.js (cost=0.00..131448.24 > rows=401 width=0) (actual time=379.407..1141.437 rows=0 loops=1) > Output: content > Filter: (((js.content -> 'tags'::text) ->> > 'title'::text) ~~ '%design%'::text) > Rows Removed by Filter: 1724810 > Buffers: shared hit=241201 > Planning time: 0.104 ms > Execution time: 403.045 ms > (14 rows) > > Time: 403.596 ms > >> or the actual time taken. >> > > Exactly what time you are referring here, Execution Time or actual time > shown on Parallel Seq Scan node and what problem do you see with > the reported time? I'm referring to the Parallel Seq Scan actual time, showing "379.407..1141.437" with 1 worker, but the total execution time shows 403.045. If one worker is taking over a second, how come the whole query was less than half a second? Thom
>
> On Thu, Nov 12, 2015 at 10:39 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > The number of shared buffers hit could be different across different runs
> > because the read sequence of parallel workers can't be guaranteed, also
> > I don't think same is even guaranteed for Seq Scan node,
>
> The number of hits could be different. However, it seems like any
> sequential scan, parallel or not, should have a number of accesses
> (hit + read) equal to the size of the relation. Not sure if that's
> what is happening here.
>
>
> >
> > The number of shared buffers hit could be different across different runs
> > because the read sequence of parallel workers can't be guaranteed, also
> > I don't think same is even guaranteed for Seq Scan node, the other
> > operations
> > in parallel could lead to different number, however the actual problem was
> > that in one of the plans shown by you [1], the Buffers hit at Gather node
> > (175288) is lesser than the Buffers hit at Parallel Seq Scan node (241201).
> > Do you still (after applying above patch) see that Gather node is showing
> > lesser hit buffers than Parallel Seq Scan node?
>
> Hmm... that's odd, I'm not seeing the problem now, so maybe I'm mistaken there.
>
Thanks for confirming the same.
> >
> >> or the actual time taken.
> >>
> >
> > Exactly what time you are referring here, Execution Time or actual time
> > shown on Parallel Seq Scan node and what problem do you see with
> > the reported time?
>
> I'm referring to the Parallel Seq Scan actual time, showing
> "379.407..1141.437" with 1 worker, but the total execution time shows
> 403.045. If one worker is taking over a second, how come the whole
> query was less than half a second?
>
Yeah, this could be possible due to the way currently time is accumulated,
yes - the another little bit unclean in EXPLAIN is number of workers. If I understand to the behave, the query is processed by two processes if workers in the explain is one.
On 13 November 2015 at 13:38, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Wed, Nov 11, 2015 at 11:40 PM, Pavel Stehule <pavel.stehule@gmail.com> > wrote: >> >> >> yes - the another little bit unclean in EXPLAIN is number of workers. If I >> understand to the behave, the query is processed by two processes if workers >> in the explain is one. >> > > You are right and I think that is current working model of Gather > node which seems okay. I think the more serious thing here > is that there is possibility that Explain Analyze can show the > number of workers as more than actual workers working for Gather > node. We have already discussed that Explain Analyze should > the actual number of workers used in query execution, patch for > the same is still pending. This may have already been discussed before, but in a verbose output, would it be possible to see the nodes for each worker? e.g. # explain (analyse, buffers, timing, verbose, costs) select count(*) from js where content->'tags'->>'title' like '%de%'; QUERY PLAN --------------------------------------------------------------------------------------------------------------------------------------Aggregate (cost=105557.59..105557.60 rows=1 width=0) (actual time=400.752..400.752 rows=1 loops=1) Output: count(*) Buffers: shared hit=175333 -> Gather (cost=1000.00..104931.04rows=250621 width=0) (actual time=400.748..400.748 rows=0 loops=1) Output: content Number of Workers: 2 Buffers: shared hit=175333 -> Parallel Seq Scan on public.js (cost=0.00..39434.47 rows=125310 width=0) (actual time=182.256..398.14 rows=0 loops=1) Output: content Filter: (((js.content-> 'tags'::text) ->> 'title'::text) ~~ '%de%'::text) Rows Removed by Filter: 626486 Buffers: shared hit=87666 -> Parallel Seq Scan on public.js (cost=0.00..39434.47 rows=1253101 width=0) (actual time=214.11..325.31 rows=0 loops=1) Output: content Filter: (((js.content-> 'tags'::text) ->> 'title'::text) ~~ '%de%'::text) Rows Removed by Filter: 6264867 Buffers: shared hit=876667Planningtime: 0.085 msExecution time: 414.713 ms (14 rows) And perhaps associated PIDs? Thom
>
> On 13 November 2015 at 13:38, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Wed, Nov 11, 2015 at 11:40 PM, Pavel Stehule <pavel.stehule@gmail.com>
> > wrote:
> >>
> >>
> >> yes - the another little bit unclean in EXPLAIN is number of workers. If I
> >> understand to the behave, the query is processed by two processes if workers
> >> in the explain is one.
> >>
> >
> > You are right and I think that is current working model of Gather
> > node which seems okay. I think the more serious thing here
> > is that there is possibility that Explain Analyze can show the
> > number of workers as more than actual workers working for Gather
> > node. We have already discussed that Explain Analyze should
> > the actual number of workers used in query execution, patch for
> > the same is still pending.
>
> This may have already been discussed before, but in a verbose output,
> would it be possible to see the nodes for each worker?
>
There will be hardly any difference in nodes for each worker and it could
>
> And perhaps associated PIDs?
>
Yeah, that can be useful, if others also feel like it is important, I can
On 13 November 2015 at 15:22, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Nov 13, 2015 at 7:59 PM, Thom Brown <thom@linux.com> wrote: >> >> On 13 November 2015 at 13:38, Amit Kapila <amit.kapila16@gmail.com> wrote: >> > On Wed, Nov 11, 2015 at 11:40 PM, Pavel Stehule >> > <pavel.stehule@gmail.com> >> > wrote: >> >> >> >> >> >> yes - the another little bit unclean in EXPLAIN is number of workers. >> >> If I >> >> understand to the behave, the query is processed by two processes if >> >> workers >> >> in the explain is one. >> >> >> > >> > You are right and I think that is current working model of Gather >> > node which seems okay. I think the more serious thing here >> > is that there is possibility that Explain Analyze can show the >> > number of workers as more than actual workers working for Gather >> > node. We have already discussed that Explain Analyze should >> > the actual number of workers used in query execution, patch for >> > the same is still pending. >> >> This may have already been discussed before, but in a verbose output, >> would it be possible to see the nodes for each worker? >> > > There will be hardly any difference in nodes for each worker and it could > be very long plan for large number of workers. What kind of additional > information you want which can't be shown in current format. For explain plans, not that useful, but it's useful to see how long each worker took for explain analyse. And I imagine as more functionality is added to scan partitions and foreign scans, it will perhaps be more useful when the plans won't be identical. (or would they?) >> >> And perhaps associated PIDs? >> > > Yeah, that can be useful, if others also feel like it is important, I can > look into preparing a patch for the same. Thanks. Thom
On Wed, Nov 11, 2015 at 6:53 AM, Robert Haas <robertmhaas@gmail.com> wrote: > > I've committed most of this, except for some planner bits that I > didn't like, and after a bunch of cleanup. Instead, I committed the > consider-parallel-v2.patch with some additional planner bits to make > up for the ones I removed from your patch. So, now we have parallel > sequential scan! Pretty cool. All I had to do is mark my slow plperl functions as being parallel safe, and bang, parallel execution of them for seq scans. But, there does seem to be a memory leak. The setup (warning: 20GB of data): create table foobar as select md5(floor(random()*1500000)::text) as id, random() as volume from generate_series(1,200000000); set max_parallel_degree TO 8; explain select count(*) from foobar where volume >0.9; QUERY PLAN ---------------------------------------------------------------------------------------Aggregate (cost=2626202.44..2626202.45rows=1 width=0) -> Gather (cost=1000.00..2576381.76 rows=19928272 width=0) Number ofWorkers: 7 -> Parallel Seq Scan on foobar (cost=0.00..582554.56 rows=19928272 width=0) Filter: (volume > '0.9'::double precision) Now running this query leads to an OOM condition: explain (analyze, buffers) select count(*) from foobar where volume >0.9; WARNING: terminating connection because of crash of another server process Running it without the explain also causes the problem. Memory dump looks like at some point before the crash looks like: TopMemoryContext: 62496 total in 9 blocks; 16976 free (60 chunks); 45520 used TopTransactionContext: 8192 total in 1 blocks;4024 free (8 chunks); 4168 used ExecutorState: 1795153920 total in 223 blocks; 4159872 free (880 chunks); 1790994048 used ExprContext: 0 total in 0 blocks; 0 free (0 chunks); 0 used Operator class cache: 8192 totalin 1 blocks; 1680 free (0 chunks); 6512 used ....other insignificant stuff... I don't have enough RAM for each of 7 workers to use all that much more than 2GB work_mem is 25MB, maintenance work_mem is 64MB Cheers, Jeff
>
> On 13 November 2015 at 15:22, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >
> > There will be hardly any difference in nodes for each worker and it could
> > be very long plan for large number of workers. What kind of additional
> > information you want which can't be shown in current format.
>
> For explain plans, not that useful, but it's useful to see how long
> each worker took for explain analyse.
>
> On Wed, Nov 11, 2015 at 6:53 AM, Robert Haas <robertmhaas@gmail.com> wrote:
> >
> > I've committed most of this, except for some planner bits that I
> > didn't like, and after a bunch of cleanup. Instead, I committed the
> > consider-parallel-v2.patch with some additional planner bits to make
> > up for the ones I removed from your patch. So, now we have parallel
> > sequential scan!
>
> Pretty cool. All I had to do is mark my slow plperl functions as
> being parallel safe, and bang, parallel execution of them for seq
> scans.
>
> But, there does seem to be a memory leak.
>
Attachment
On Fri, Nov 13, 2015 at 10:46 AM, Thom Brown <thom@linux.com> wrote: >>> And perhaps associated PIDs? >> >> Yeah, that can be useful, if others also feel like it is important, I can >> look into preparing a patch for the same. > > Thanks. Thom, what do you think the EXPLAIN output should look like, specifically? Or anyone else who feels like answering. I don't think it would be very useful to repeat the entire EXPLAIN output n times, once per worker. That sounds like a loser. But we could add additional lines to the output for each node, like this: Parallel Seq Scan on foo (cost=0.00..XXX rows=YYY width=ZZZ) (actual time=AAA..BBB rows=CCC loops=1) Leader: actual time=AAA..BBB rows=CCC loops=1 Worker 0: actual time=AAA..BBB rows=CCC loops=1Worker 1: actual time=AAA..BBB rows=CCC loops=1 Worker 2: actual time=AAA..BBB rows=CCC loops=1 If "buffers" is specified, we could display the summary information after the Parallel Seq Scan as normal and then display an additional per-worker line after the "Leader" line and each "Worker N" line. I think displaying the worker index is more useful than displaying the PID, especially if we think that a plan tree like this might ever get executed multiple times with different PIDs on each pass. Like? Dislike? Other ideas? -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On 16/11/15 12:05, Robert Haas wrote: > On Fri, Nov 13, 2015 at 10:46 AM, Thom Brown <thom@linux.com> wrote: >>>> And perhaps associated PIDs? >>> Yeah, that can be useful, if others also feel like it is important, I can >>> look into preparing a patch for the same. >> Thanks. > Thom, what do you think the EXPLAIN output should look like, > specifically? Or anyone else who feels like answering. > > I don't think it would be very useful to repeat the entire EXPLAIN > output n times, once per worker. That sounds like a loser. But we > could add additional lines to the output for each node, like this: > > Parallel Seq Scan on foo (cost=0.00..XXX rows=YYY width=ZZZ) (actual > time=AAA..BBB rows=CCC loops=1) > Leader: actual time=AAA..BBB rows=CCC loops=1 > Worker 0: actual time=AAA..BBB rows=CCC loops=1 > Worker 1: actual time=AAA..BBB rows=CCC loops=1 > Worker 2: actual time=AAA..BBB rows=CCC loops=1 > > If "buffers" is specified, we could display the summary information > after the Parallel Seq Scan as normal and then display an additional > per-worker line after the "Leader" line and each "Worker N" line. I > think displaying the worker index is more useful than displaying the > PID, especially if we think that a plan tree like this might ever get > executed multiple times with different PIDs on each pass. > > Like? Dislike? Other ideas? > Possibly have an option to include the PID? Consider altering the format field width of the Worker number (depending on the number of workers) so you don't get: Worker 9 ... Worker 10 ... but something like Worker 9 ... Worker 10 ... Cheers, Gavin
On Sun, Nov 15, 2015 at 1:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Thanks for the report. > > I think main reason of the leak in workers seems to be due the reason > that one of the buffer used while sending tuples (in function > BuildRemapInfo) > from worker to master is not getting freed and it is allocated for each > tuple worker sends back to master. I couldn't find use of such a buffer, > so I think we can avoid the allocation of same or atleast we need to free > it. Attached patch remove_unused_buf_allocation_v1.patch should fix the > issue. Oops. Committed. > Another thing I have noticed is that we need to build the remap info > target list contains record type of attrs, so ideally it should not even go > in > this path when such attrs are not present. The reason for the same was > that the tuple descriptor stored in TQueueDestReceiver was not updated, > attached patch fix_initialization_tdesc_v1 fixes this issue. I don't understand this part. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Fri, Nov 13, 2015 at 10:46 AM, Thom Brown <thom@linux.com> wrote:
> >>> And perhaps associated PIDs?
> >>
> >> Yeah, that can be useful, if others also feel like it is important, I can
> >> look into preparing a patch for the same.
> >
> > Thanks.
>
> Thom, what do you think the EXPLAIN output should look like,
> specifically? Or anyone else who feels like answering.
>
> I don't think it would be very useful to repeat the entire EXPLAIN
> output n times, once per worker. That sounds like a loser.
On Mon, Nov 16, 2015 at 4:35 AM, Robert Haas <robertmhaas@gmail.com> wrote:
>
> On Fri, Nov 13, 2015 at 10:46 AM, Thom Brown <thom@linux.com> wrote:
> >>> And perhaps associated PIDs?
> >>
> >> Yeah, that can be useful, if others also feel like it is important, I can
> >> look into preparing a patch for the same.
> >
> > Thanks.
>
> Thom, what do you think the EXPLAIN output should look like,
> specifically? Or anyone else who feels like answering.
>
> I don't think it would be very useful to repeat the entire EXPLAIN
> output n times, once per worker. That sounds like a loser.>Yes, it doesn't seem good idea to repeat the information, but whatabout the cases when different workers perform scan on differentrelations (partitions in case of Append node) or may be performs adifferent operation in Sort or join node parallelism.
+1
On Sat, Nov 14, 2015 at 10:12 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Nov 13, 2015 at 11:05 PM, Jeff Janes <jeff.janes@gmail.com> wrote: >> >> On Wed, Nov 11, 2015 at 6:53 AM, Robert Haas <robertmhaas@gmail.com> >> wrote: >> > >> > I've committed most of this, except for some planner bits that I >> > didn't like, and after a bunch of cleanup. Instead, I committed the >> > consider-parallel-v2.patch with some additional planner bits to make >> > up for the ones I removed from your patch. So, now we have parallel >> > sequential scan! >> >> Pretty cool. All I had to do is mark my slow plperl functions as >> being parallel safe, and bang, parallel execution of them for seq >> scans. >> >> But, there does seem to be a memory leak. >> > > Thanks for the report. > > I think main reason of the leak in workers seems to be due the reason > that one of the buffer used while sending tuples (in function > BuildRemapInfo) > from worker to master is not getting freed and it is allocated for each > tuple worker sends back to master. I couldn't find use of such a buffer, > so I think we can avoid the allocation of same or atleast we need to free > it. Attached patch remove_unused_buf_allocation_v1.patch should fix the > issue. Thanks, that patch (as committed) has fixed the problem for me. I don't understand the second one. Cheers, Jeff
Nov 16 20:40:05 woludwha02 kernel: postgres[22918]: segfault at 7fa3437bf104 ip 0000000000490b56 sp 00007ffdf2f083a0 error 6 in postgres[400000+5b5000]
On Sat, Nov 14, 2015 at 10:12 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> On Fri, Nov 13, 2015 at 11:05 PM, Jeff Janes <jeff.janes@gmail.com> wrote:
>>
>> On Wed, Nov 11, 2015 at 6:53 AM, Robert Haas <robertmhaas@gmail.com>
>> wrote:
>> >
>> > I've committed most of this, except for some planner bits that I
>> > didn't like, and after a bunch of cleanup. Instead, I committed the
>> > consider-parallel-v2.patch with some additional planner bits to make
>> > up for the ones I removed from your patch. So, now we have parallel
>> > sequential scan!
>>
>> Pretty cool. All I had to do is mark my slow plperl functions as
>> being parallel safe, and bang, parallel execution of them for seq
>> scans.
>>
>> But, there does seem to be a memory leak.
>>
>
> Thanks for the report.
>
> I think main reason of the leak in workers seems to be due the reason
> that one of the buffer used while sending tuples (in function
> BuildRemapInfo)
> from worker to master is not getting freed and it is allocated for each
> tuple worker sends back to master. I couldn't find use of such a buffer,
> so I think we can avoid the allocation of same or atleast we need to free
> it. Attached patch remove_unused_buf_allocation_v1.patch should fix the
> issue.
Thanks, that patch (as committed) has fixed the problem for me. I
don't understand the second one.
Cheers,
Jeff
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
--
0477/305361
On Mon, Nov 16, 2015 at 2:51 PM, Bert <biertie@gmail.com> wrote: > I've just pulled and compiled the new code. > I'm running a TPC-DS like test on different PostgreSQL installations, but > running (max) 12queries in parallel on a server with 12cores. > I've configured max_parallel_degree to 2, and I get messages that backend > processes crash. > I am running the same test now with 6queries in parallel, and parallel > degree to 2, and they seem to work. for now. :) > > This is the output I get in /var/log/messages > Nov 16 20:40:05 woludwha02 kernel: postgres[22918]: segfault at 7fa3437bf104 > ip 0000000000490b56 sp 00007ffdf2f083a0 error 6 in postgres[400000+5b5000] > > Is there something else I should get? Can you enable core dumps e.g. by passing the -c option to pg_ctl start? If you can get a core file, you can then get a backtrace using: gdb /path/to/postgres /path/to/core bt full q That should be enough to find and fix whatever the bug is. Thanks for testing. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
> On Sun, Nov 15, 2015 at 1:12 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Thanks for the report.
> >
> > I think main reason of the leak in workers seems to be due the reason
> > that one of the buffer used while sending tuples (in function
> > BuildRemapInfo)
> > from worker to master is not getting freed and it is allocated for each
> > tuple worker sends back to master. I couldn't find use of such a buffer,
> > so I think we can avoid the allocation of same or atleast we need to free
> > it. Attached patch remove_unused_buf_allocation_v1.patch should fix the
> > issue.
>
> Oops. Committed.
>
> > Another thing I have noticed is that we need to build the remap info
> > target list contains record type of attrs, so ideally it should not even go
> > in
> > this path when such attrs are not present. The reason for the same was
> > that the tuple descriptor stored in TQueueDestReceiver was not updated,
> > attached patch fix_initialization_tdesc_v1 fixes this issue.
>
> I don't understand this part.
>
tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
if (tqueue->tupledesc != tupledesc ||
tqueue->remapinfo->natts != tupledesc->natts)
{
if (tqueue->remapinfo != NULL)
pfree(tqueue->remapinfo);
tqueue->remapinfo = BuildRemapInfo(tupledesc);
}
>
> On Sat, Nov 14, 2015 at 10:12 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Fri, Nov 13, 2015 at 11:05 PM, Jeff Janes <jeff.janes@gmail.com> wrote:
> >
> > I think main reason of the leak in workers seems to be due the reason
> > that one of the buffer used while sending tuples (in function
> > BuildRemapInfo)
> > from worker to master is not getting freed and it is allocated for each
> > tuple worker sends back to master. I couldn't find use of such a buffer,
> > so I think we can avoid the allocation of same or atleast we need to free
> > it. Attached patch remove_unused_buf_allocation_v1.patch should fix the
> > issue.
>
> Thanks, that patch (as committed) has fixed the problem for me.
>
gdb /var/lib/pgsql/9.6/data/ /var/lib/pgsql/9.6/data/core.7877
GNU gdb (GDB) Red Hat Enterprise Linux 7.6.1-64.el7
Copyright (C) 2013 Free Software Foundation, Inc.
License GPLv3+: GNU GPL version 3 or later <http://gnu.org/licenses/gpl.html>
This is free software: you are free to change and redistribute it.
There is NO WARRANTY, to the extent permitted by law. Type "show copying"
and "show warranty" for details.
This GDB was configured as "x86_64-redhat-linux-gnu".
For bug reporting instructions, please see:
<http://www.gnu.org/software/gdb/bugs/>...
/var/lib/pgsql/9.6/data/: Success.
[New LWP 7877]
Missing separate debuginfo for the main executable file
Try: yum --enablerepo='*debug*' install /usr/lib/debug/.build-id/02/20b77a9ab8f607b0610082794165fccedf210d
Core was generated by `postgres: postgres tpcds [loca'.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000490b56 in ?? ()
(gdb) bt full
#0 0x0000000000490b56 in ?? ()
No symbol table info available.
#1 0x0000000000003668 in ?? ()
No symbol table info available.
#2 0x00007f956249a008 in ?? ()
No symbol table info available.
#3 0x000000000228c498 in ?? ()
No symbol table info available.
#4 0x0000000000000001 in ?? ()
No symbol table info available.
#5 0x000000000228ad00 in ?? ()
No symbol table info available.
#6 0x0000000000493fdf in ?? ()
No symbol table info available.
#7 0x00000000021a8e50 in ?? ()
No symbol table info available.
#8 0x0000000000000000 in ?? ()
No symbol table info available.
(gdb) q
On Mon, Nov 16, 2015 at 2:51 PM, Bert <biertie@gmail.com> wrote:
> I've just pulled and compiled the new code.
> I'm running a TPC-DS like test on different PostgreSQL installations, but
> running (max) 12queries in parallel on a server with 12cores.
> I've configured max_parallel_degree to 2, and I get messages that backend
> processes crash.
> I am running the same test now with 6queries in parallel, and parallel
> degree to 2, and they seem to work. for now. :)
>
> This is the output I get in /var/log/messages
> Nov 16 20:40:05 woludwha02 kernel: postgres[22918]: segfault at 7fa3437bf104
> ip 0000000000490b56 sp 00007ffdf2f083a0 error 6 in postgres[400000+5b5000]
>
> Is there something else I should get?
Can you enable core dumps e.g. by passing the -c option to pg_ctl
start? If you can get a core file, you can then get a backtrace
using:
gdb /path/to/postgres /path/to/core
bt full
q
That should be enough to find and fix whatever the bug is. Thanks for testing.
--
0477/305361
(gdb) bt full
#0 0x0000000000490b56 in heap_parallelscan_nextpage ()
No symbol table info available.
#1 0x0000000000493fdf in heap_getnext ()
No symbol table info available.
#2 0x00000000005c0733 in SeqNext ()
No symbol table info available.
#3 0x00000000005ac5d9 in ExecScan ()
No symbol table info available.
#4 0x00000000005a5c08 in ExecProcNode ()
No symbol table info available.
#5 0x00000000005b5298 in ExecGather ()
No symbol table info available.
#6 0x00000000005a5aa8 in ExecProcNode ()
No symbol table info available.
#7 0x00000000005b68b9 in MultiExecHash ()
No symbol table info available.
#8 0x00000000005b7256 in ExecHashJoin ()
No symbol table info available.
#9 0x00000000005a5b18 in ExecProcNode ()
No symbol table info available.
#10 0x00000000005b0ac9 in fetch_input_tuple ()
No symbol table info available.
#11 0x00000000005b1eaf in ExecAgg ()
No symbol table info available.
#12 0x00000000005a5ad8 in ExecProcNode ()
No symbol table info available.
#13 0x00000000005c11e1 in ExecSort ()
No symbol table info available.
#14 0x00000000005a5af8 in ExecProcNode ()
No symbol table info available.
#15 0x00000000005ba164 in ExecLimit ()
No symbol table info available.
#16 0x00000000005a5a38 in ExecProcNode ()
No symbol table info available.
#17 0x00000000005a2343 in standard_ExecutorRun ()
No symbol table info available.
#18 0x000000000069cb08 in PortalRunSelect ()
No symbol table info available.
#19 0x000000000069de5f in PortalRun ()
No symbol table info available.
#20 0x000000000069bc16 in PostgresMain ()
No symbol table info available.
#21 0x0000000000466f55 in ServerLoop ()
No symbol table info available.
#22 0x0000000000648436 in PostmasterMain ()
No symbol table info available.
#23 0x00000000004679f0 in main ()
No symbol table info available.
Is there something else I can do?Hi,this is the backtrace:
gdb /var/lib/pgsql/9.6/data/ /var/lib/pgsql/9.6/data/core.7877
GNU gdb (GDB) Red Hat Enterprise Linux 7.6.1-64.el7
Copyright (C) 2013 Free Software Foundation, Inc.
License GPLv3+: GNU GPL version 3 or later <http://gnu.org/licenses/gpl.html>
This is free software: you are free to change and redistribute it.
There is NO WARRANTY, to the extent permitted by law. Type "show copying"
and "show warranty" for details.
This GDB was configured as "x86_64-redhat-linux-gnu".
For bug reporting instructions, please see:
<http://www.gnu.org/software/gdb/bugs/>...
/var/lib/pgsql/9.6/data/: Success.
[New LWP 7877]
Missing separate debuginfo for the main executable file
Try: yum --enablerepo='*debug*' install /usr/lib/debug/.build-id/02/20b77a9ab8f607b0610082794165fccedf210d
Core was generated by `postgres: postgres tpcds [loca'.
Program terminated with signal 11, Segmentation fault.
#0 0x0000000000490b56 in ?? ()
(gdb) bt full
#0 0x0000000000490b56 in ?? ()
No symbol table info available.
#1 0x0000000000003668 in ?? ()
No symbol table info available.
#2 0x00007f956249a008 in ?? ()
No symbol table info available.
#3 0x000000000228c498 in ?? ()
No symbol table info available.
#4 0x0000000000000001 in ?? ()
No symbol table info available.
#5 0x000000000228ad00 in ?? ()
No symbol table info available.
#6 0x0000000000493fdf in ?? ()
No symbol table info available.
#7 0x00000000021a8e50 in ?? ()
No symbol table info available.
#8 0x0000000000000000 in ?? ()
No symbol table info available.
(gdb) q--On Mon, Nov 16, 2015 at 8:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:On Mon, Nov 16, 2015 at 2:51 PM, Bert <biertie@gmail.com> wrote:
> I've just pulled and compiled the new code.
> I'm running a TPC-DS like test on different PostgreSQL installations, but
> running (max) 12queries in parallel on a server with 12cores.
> I've configured max_parallel_degree to 2, and I get messages that backend
> processes crash.
> I am running the same test now with 6queries in parallel, and parallel
> degree to 2, and they seem to work. for now. :)
>
> This is the output I get in /var/log/messages
> Nov 16 20:40:05 woludwha02 kernel: postgres[22918]: segfault at 7fa3437bf104
> ip 0000000000490b56 sp 00007ffdf2f083a0 error 6 in postgres[400000+5b5000]
>
> Is there something else I should get?
Can you enable core dumps e.g. by passing the -c option to pg_ctl
start? If you can get a core file, you can then get a backtrace
using:
gdb /path/to/postgres /path/to/core
bt full
q
That should be enough to find and fix whatever the bug is. Thanks for testing.Bert Desmet
0477/305361
--
0477/305361
On Mon, Nov 16, 2015 at 9:49 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I don't understand this part. >> > > The code in question is as below: > > tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self) > > { > .. > > if (tqueue->tupledesc != tupledesc || > > tqueue->remapinfo->natts != tupledesc->natts) > > { > > if (tqueue->remapinfo != NULL) > > pfree(tqueue->remapinfo); > > tqueue->remapinfo = BuildRemapInfo(tupledesc); > > } > > .. > } > > Here the above check always passes as tqueue->tupledesc is not > set due to which it always try to build remap info. Is there any reason > for doing so? Groan. The problem here is that tqueue->tupledesc never gets set. I think this should be fixed as in the attached. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
Attachment
On Tue, Nov 17, 2015 at 6:52 AM, Bert <biertie@gmail.com> wrote: > edit: maybe this is more useful? :) Definitely. But if you've built with --enable-debug and not stripped the resulting executable, we ought to get line numbers as well, plus the arguments to each function on the stack. That would help a lot more. The only things that get dereferenced in that function are "scan" and "parallel_scan", so it's a good bet that one of those pointers is pointing off into never-never land. I can't immediately guess how that's happening, though. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Thu, Nov 12, 2015 at 10:23 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Thanks for the report. The reason for this problem is that instrumentation > information from workers is getting aggregated multiple times. In > ExecShutdownGatherWorkers(), we call ExecParallelFinish where it > will wait for workers to finish and then accumulate stats from workers. > Now ExecShutdownGatherWorkers() could be called multiple times > (once we read all tuples from workers, at end of node) and it should be > ensured that repeated calls should not try to redo the work done by first > call. > The same is ensured for tuplequeues, but not for parallel executor info. > I think we can safely assume that we need to call ExecParallelFinish() only > when there are workers started by the Gathers node, so on those lines > attached patch should fix the problem. I suggest that we instead fix ExecParallelFinish() to be idempotent. Add a "bool finished" flag to ParallelExecutorInfo and return at once if it's already set. Get rid of the exposed ExecParallelReinitializeTupleQueues() interface and have ExecParallelReinitialize(pei) instead. Have that call ReinitializeParallelDSM(), ExecParallelSetupTupleQueues(pei->pcxt, true), and set pei->finished = false. I think that would give us a slightly cleaner separation of concerns between nodeGather.c and execParallel.c. Your fix seems a little fragile. You're relying on node->reader != NULL to tell you whether the readers need to be cleaned up, but in fact node->reader is set to a non-NULL value AFTER the pei has been created. Granted, we currently always create a reader unless we don't get any workers, and if we don't get any workers then failing to call ExecParallelFinish is currently harmless, but nonetheless I think we should be more explicit about this so it doesn't accidentally get broken later. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
On Tue, Nov 17, 2015 at 6:52 AM, Bert <biertie@gmail.com> wrote:
> edit: maybe this is more useful? :)
Definitely. But if you've built with --enable-debug and not stripped
the resulting executable, we ought to get line numbers as well, plus
the arguments to each function on the stack. That would help a lot
more. The only things that get dereferenced in that function are
"scan" and "parallel_scan", so it's a good bet that one of those
pointers is pointing off into never-never land. I can't immediately
guess how that's happening, though.
--
0477/305361
anyhow, find attached a third attempt to a valid backtrace file.Hey Robert,Thank you for the help. As you might (not) know, I'm quite new to the community, but I'm learning. with the help from people like you.This run is compiled from commit 5f10b7a604c87fc61a2c20a56552301f74c9bd5f and your latest patch atteched in this mailtrack.
>
> On Mon, Nov 16, 2015 at 9:49 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> >> I don't understand this part.
> >>
> >
> > Here the above check always passes as tqueue->tupledesc is not
> > set due to which it always try to build remap info. Is there any reason
> > for doing so?
>
> Groan. The problem here is that tqueue->tupledesc never gets set.
> think this should be fixed as in the attached.
>
>
> On Thu, Nov 12, 2015 at 10:23 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Thanks for the report. The reason for this problem is that instrumentation
> > information from workers is getting aggregated multiple times. In
> > ExecShutdownGatherWorkers(), we call ExecParallelFinish where it
> > will wait for workers to finish and then accumulate stats from workers.
> > Now ExecShutdownGatherWorkers() could be called multiple times
> > (once we read all tuples from workers, at end of node) and it should be
> > ensured that repeated calls should not try to redo the work done by first
> > call.
> > The same is ensured for tuplequeues, but not for parallel executor info.
> > I think we can safely assume that we need to call ExecParallelFinish() only
> > when there are workers started by the Gathers node, so on those lines
> > attached patch should fix the problem.
>
> I suggest that we instead fix ExecParallelFinish() to be idempotent.
> Add a "bool finished" flag to ParallelExecutorInfo and return at once
> if it's already set. Get rid of the exposed
> ExecParallelReinitializeTupleQueues() interface and have
> ExecParallelReinitialize(pei) instead. Have that call
> ReinitializeParallelDSM(), ExecParallelSetupTupleQueues(pei->pcxt,
> true), and set pei->finished = false. I think that would give us a
> slightly cleaner separation of concerns between nodeGather.c and
> execParallel.c.
>
Okay, attached patch fixes the issue as per above suggestion.
Attachment
On Wed, Nov 18, 2015 at 12:48 AM, Amit Kapila <amit.kapila16@gmail.com> wrote: >> I suggest that we instead fix ExecParallelFinish() to be idempotent. >> Add a "bool finished" flag to ParallelExecutorInfo and return at once >> if it's already set. Get rid of the exposed >> ExecParallelReinitializeTupleQueues() interface and have >> ExecParallelReinitialize(pei) instead. Have that call >> ReinitializeParallelDSM(), ExecParallelSetupTupleQueues(pei->pcxt, >> true), and set pei->finished = false. I think that would give us a >> slightly cleaner separation of concerns between nodeGather.c and >> execParallel.c. > > Okay, attached patch fixes the issue as per above suggestion. Thanks, committed. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
This is the output I get in /var/log/messagesI am running the same test now with 6queries in parallel, and parallel degree to 2, and they seem to work. for now. :)I've configured max_parallel_degree to 2, and I get messages that backend processes crash.I'm running a TPC-DS like test on different PostgreSQL installations, but running (max) 12queries in parallel on a server with 12cores.Hey,I've just pulled and compiled the new code.
Nov 16 20:40:05 woludwha02 kernel: postgres[22918]: segfault at 7fa3437bf104 ip 0000000000490b56 sp 00007ffdf2f083a0 error 6 in postgres[400000+5b5000]
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachment
On Wed, Nov 18, 2015 at 10:41 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > I think whats going on here is that when any of the session doesn't > get any workers, we shutdown the Gather node which internally destroys > the dynamic shared memory segment as well. However the same is > needed as per current design for doing scan by master backend as > well. So I think the fix would be to just do shutdown of workers which > actually won't do anything in this scenario. It seems silly to call ExecGatherShutdownWorkers() here when that's going to be a no-op. I think we should just remove that line and the if statement before it altogether and replace it with a comment explaining why we can't nuke the DSM at this stage. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Wed, Nov 18, 2015 at 10:41 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > I think whats going on here is that when any of the session doesn't
> > get any workers, we shutdown the Gather node which internally destroys
> > the dynamic shared memory segment as well. However the same is
> > needed as per current design for doing scan by master backend as
> > well. So I think the fix would be to just do shutdown of workers which
> > actually won't do anything in this scenario.
>
> It seems silly to call ExecGatherShutdownWorkers() here when that's
> going to be a no-op. I think we should just remove that line and the
> if statement before it altogether and replace it with a comment
> explaining why we can't nuke the DSM at this stage.
>
Attachment
On Thu, Nov 19, 2015 at 11:59 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > Isn't it better to destroy the memory for readers array as that gets > allocated > even if there are no workers available for execution? > > Attached patch fixes the issue by just destroying readers array. Well, then you're making ExecGatherShutdownWorkers() not a no-op any more. I'll go commit a combination of your two patches. -- Robert Haas EnterpriseDB: http://www.enterprisedb.com The Enterprise PostgreSQL Company
>
> On Thu, Nov 19, 2015 at 11:59 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > Isn't it better to destroy the memory for readers array as that gets
> > allocated
> > even if there are no workers available for execution?
> >
> > Attached patch fixes the issue by just destroying readers array.
>
> Well, then you're making ExecGatherShutdownWorkers() not a no-op any
> more. I'll go commit a combination of your two patches.
>
On Sun, Nov 22, 2015 at 3:25 PM, Amit Kapila <amit.kapila16@gmail.com> wrote: > On Fri, Nov 20, 2015 at 11:34 PM, Robert Haas <robertmhaas@gmail.com> wrote: >> >> On Thu, Nov 19, 2015 at 11:59 PM, Amit Kapila <amit.kapila16@gmail.com> >> wrote: >> > Isn't it better to destroy the memory for readers array as that gets >> > allocated >> > even if there are no workers available for execution? >> > >> > Attached patch fixes the issue by just destroying readers array. >> >> Well, then you're making ExecGatherShutdownWorkers() not a no-op any >> more. I'll go commit a combination of your two patches. >> > > Thanks! There is still an entry in the CF app for this thread as "Parallel Seq scan". The basic infrastructure has been committed, and I understand that this is a never-ending tasks and that there will be many optimizations. Still, are you guys fine to switch this entry as committed for now? -- Michael
>
> On Sun, Nov 22, 2015 at 3:25 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
> > On Fri, Nov 20, 2015 at 11:34 PM, Robert Haas <robertmhaas@gmail.com> wrote:
> >>
> >> On Thu, Nov 19, 2015 at 11:59 PM, Amit Kapila <amit.kapila16@gmail.com>
> >> wrote:
> >> > Isn't it better to destroy the memory for readers array as that gets
> >> > allocated
> >> > even if there are no workers available for execution?
> >> >
> >> > Attached patch fixes the issue by just destroying readers array.
> >>
> >> Well, then you're making ExecGatherShutdownWorkers() not a no-op any
> >> more. I'll go commit a combination of your two patches.
> >>
> >
> > Thanks!
>
> There is still an entry in the CF app for this thread as "Parallel Seq
> scan". The basic infrastructure has been committed, and I understand
> that this is a never-ending tasks and that there will be many
> optimizations. Still, are you guys fine to switch this entry as
> committed for now?
>
On Wed, Dec 2, 2015 at 5:45 PM, Amit Kapila wrote: > I am fine with it. I think the further optimizations can be done > separately. Done. -- Michael