Parallel Seq Scan
As per discussion on another thread related to using
custom scan nodes for prototype of parallel sequence scan,
I have developed the same, but directly by adding
new nodes for parallel sequence scan. There might be
some advantages for developing this as a contrib
module by using custom scan nodes, however I think
we might get stucked after some point due to custom
scan node capability as pointed out by Andres.
The basic idea used is that while evaluating the cheapest
path for scan, optimizer will also evaluate if it can use
parallel seq path. Currently I have kept a very simple
model to calculate the cost of parallel sequence path which
is that divide the cost for CPU and disk by availble number
of worker backends (We can enhance it based on further
experiments and discussion; we need to consider worker startup
and dynamic shared memory setup cost as well). The work aka
scan of blocks is divided equally among all workers (except for
corner cases where blocks can't be equally divided among workers,
the last worker will be responsible for scanning the remaining blocks).
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.
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.
Now to fetch the data from multiple queues corresponding to each
worker a simple mechanism is used that is fetch from first queue
till all the data is consumed from same, then fetch from second
queue and so on. Also here master backend is responsible for just
getting the data from workers and passing it back to client.
I am sure that we can improve this strategy in many ways
like by making master backend to also perform scan for some
of the blocks rather than just getting data from workers and
a better strategy to fetch the data from multiple queues.
Worker backend will receive the information related to scan
from master backend and generate the plan from same and
execute that plan, so here the work to scan the data after
generating the plan is very much similar to exec_simple_query()
(i.e Create the portal and run it based on planned statement)
except that worker backends will initialize the block range it want to
scan in executor initialization phase (ExecInitSeqScan()).
Workers will exit after sending the data to master backend
which essentially means that for each execution we need
to initiate the workers, I think here we can improve by giving the
control for workers to postmaster so that we don't need to
initialize them each time during execution, however this can
be a totally separate optimization which is better to be done
independently of this patch.
As currently we don't have mechanism to share transaction
state, I have used separate transaction in worker backend to
execute the plan.
Any error in master backend either via backend worker or due
to other issue in master backend itself should terminate all the
workers before aborting the transaction.
We can't do it with the error context callback mechanism
(error_context_stack) which we use at other places in code, as
for this case we need it from the time workers are started till
the execution is complete (error_context_stack could get reset
once the control goes out of the function which has set it.)
One way could be that maintain the callback information in
TransactionState and use it to kill the workers before aborting
transaction in main backend. Another could be that have another
variable similar to error_context_stack (which will be used
specifically for storing the workers state), and kill the workers
in errfinish via callback. Currently I have handled it at the time of
detaching from shared memory.
Another point that needs to be taken care in worker backend is
that if any error occurs, we should *not* abort the transaction as
the transaction state is shared across all workers.
Currently the parallel seq scan will not be considered
for statements other than SELECT or if there is a join in
the statement or if statement contains quals or if target
list contains non-Var fields. We can definitely support
simple quals and targetlist other than non-Vars. By simple,
I means that it should not contain functions or some other
conditions which can't be pushed down to worker backend.
Behaviour of some simple statements with patch is as below:
postgres=# create table t1(c1 int, c2 char(500)) with (fillfactor=10);
CREATE TABLE
postgres=# insert into t1 values(generate_series(1,100),'amit');
INSERT 0 100
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)
postgres=# explain select Distinct(c1) from t1;
QUERY PLAN
--------------------------------------------------------------------
HashAggregate (cost=25.50..26.50 rows=100 width=4)
Group Key: c1
-> Parallel Seq Scan on t1 (cost=0.00..25.25 rows=100 width=4)
Number of Workers: 4
Number of Blocks Per Workers: 25
(5 rows)
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v1.patchapplication/octet-stream; name=parallel_seqscan_v1.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..7be7ba8
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,374 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg);
+
+/*
+ * Indicate that an error came from a particular worker.
+ */
+static void
+worker_error_callback(void *arg)
+{
+ pid_t pid = * (pid_t *) arg;
+
+ errcontext("worker backend, pid %d", pid);
+}
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc)
+{
+ shm_mq_result res;
+ char msgtype;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int32 pid = 1234;
+ int queueId = 0;
+
+
+ /*state = palloc0(sizeof(worker_result_state));
+ state->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ state->typioparams = palloc(sizeof(Oid) * natts);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &state->typioparams[i]);
+ fmgr_info(receive_function_id, &state->receive_functions[i]);
+ }*/
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ --shmScan->num_shm_queues;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Initialize message buffer. */
+ initStringInfo(&msg);
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, false);
+ if (res != SHM_MQ_SUCCESS)
+ break;
+
+ /*
+ * Message-parsing routines operate on a null-terminated StringInfo,
+ * so we must construct one.
+ */
+ resetStringInfo(&msg);
+ enlargeStringInfo(&msg, nbytes);
+ msg.len = nbytes;
+ memcpy(msg.data, data, nbytes);
+ msg.data[nbytes] = '\0';
+ msgtype = pq_getmsgbyte(&msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback context;
+
+ /* Parse ErrorResponse or NoticeResponse. */
+ pq_parse_errornotice(&msg, &edata);
+
+ /*
+ * Limit the maximum error level to ERROR. We don't want
+ * a FATAL inside the backend worker to kill the user
+ * session.
+ */
+ if (edata.elevel > ERROR)
+ edata.elevel = ERROR;
+
+ /*
+ * Rethrow the error with an appropriate context method.
+ * On error, we need to ensure that master backend stop
+ * all other workers before propagating the error, so
+ * we need to pass the pid's of all workers, so that same
+ * can be done in error callback.
+ * XXX - For now, I am just sending some random number, this
+ * needs to be fixed.
+ */
+ context.callback = worker_error_callback;
+ context.arg = (void *) &pid;
+ context.previous = error_context_stack;
+ error_context_stack = &context;
+ ThrowErrorData(&edata);
+ error_context_stack = context.previous;
+
+ break;
+ }
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg.data[0], &msg.data[1], nbytes - 1);
+ break;
+ }
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(&msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description)
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(&msg); /* name */
+ (void) pq_getmsgint(&msg, 4); /* table OID */
+ (void) pq_getmsgint(&msg, 2); /* table attnum */
+ type_id = pq_getmsgint(&msg, 4); /* type OID */
+ (void) pq_getmsgint(&msg, 2); /* type length */
+ (void) pq_getmsgint(&msg, 4); /* typmod */
+ (void) pq_getmsgint(&msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(&msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ HeapTuple result;
+
+ result = form_result_tuple(resultState, tupdesc, &msg);
+ return result;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(&msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+
+ case 'Z':
+ {
+ /* Handle ReadyForQuery message. */
+ resultState->complete = true;
+ break;
+ }
+ default:
+ elog(WARNING, "unknown message type: %c (%zu bytes)",
+ msg.data[0], nbytes);
+ break;
+ }
+ }
+
+ /* Check whether the connection was broken prematurely. */
+ if (!resultState->complete)
+ ereport(ERROR,
+ (errcode(ERRCODE_CONNECTION_FAILURE),
+ errmsg("lost connection to worker process with PID %d",
+ pid)));
+
+ /*
+ * if we have exhausted data from all worker queues, then terminate
+ * processing data from queues.
+ */
+ if (shmScan->num_shm_queues <=0)
+ break;
+ else
+ {
+ ++queueId;
+ --shmScan->num_shm_queues;
+ resultState->has_row_description = false;
+ }
+ }
+
+ return NULL;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description)
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ return heap_form_tuple(tupdesc, values, isnull);
+}
\ No newline at end of file
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 332f04a..f158583 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -714,6 +714,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -910,6 +911,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1059,6 +1063,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1325,6 +1330,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Workers",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2142,6 +2157,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index e27c062..a28e74e 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..3d651b5
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,441 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * SeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(node->pss_currentShmScanDesc, node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not check
+ * are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(parallelscanstate->ss.ss_currentRelation->rd_id,
+ node->scan.plan.targetlist,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->seg,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+
+ /*
+ * use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(node->num_workers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ /* detach from dynamic shared memory. */
+ dsm_detach(node->seg);
+}
+
+/*
+ * EstimateScanRelationIdSpace:
+ * Returns the size needed to store the ScanRelaionId for the current query
+ */
+Size
+EstimateScanRelationIdSpace(Oid relId)
+{
+ Size size;
+
+ size = sizeof(relId);
+
+ return size;
+}
+
+/*
+ * SerializeScanRelationId:
+ * Dumps the relatinId onto the memory location at start_address.
+ */
+void
+SerializeScanRelationId(Oid relId, Size maxsize, char *start_address)
+{
+ memcpy(start_address, &relId, maxsize);
+}
+
+/*
+ * RestoreScanRelationId:
+ * Reads the relationId from the specified address, restore it into given
+ * relationId.
+ */
+void
+RestoreScanRelationId(Oid *relId, char *start_address)
+{
+ memcpy(relId, start_address, sizeof(Oid));
+}
+
+/*
+ * EstimateTargetListSpace:
+ * Returns the size needed to store the Targetlist for the current query
+ */
+Size
+EstimateTargetListSpace(List *targetList)
+{
+ Size size;
+
+ /* Add space reqd for saving the data size of the targetlist */
+ size = sizeof(Size);
+
+ size = add_size(size,
+ mul_size(targetList->length, sizeof(TargetEntry)));
+
+ /*
+ * For now, lets just support for Var type of nodes.
+ *
+ * FIXME - we need to traverse target list and allocate depending
+ * on the node type.
+ */
+ /*size = add_size(size, mul_size(targetList->length, sizeof(Expr)));*/
+ size = add_size(size, mul_size(targetList->length, sizeof(Var)));
+
+ /*
+ * Account for column names, we could get exact length for each column
+ * name, however as NAMEDATALEN is not too big, this seems okay.
+ */
+ size = add_size(size, mul_size(targetList->length, NAMEDATALEN));
+
+ return size;
+}
+
+/*
+ * SerializeTargetList:
+ * Dumps the each target entry onto the memory location at start_address.
+ */
+void
+SerializeTargetList(List *targetList, Size maxsize, char *start_address)
+{
+ Size targetListSize;
+ char *curptr;
+ ListCell *l;
+ TargetEntry *srctargetEntry;
+ TargetEntry *desttargetEntry;
+
+ targetListSize = targetList->length;
+
+ /* copy target list size */
+ memcpy(start_address, &targetListSize, sizeof(targetListSize));
+ curptr = start_address + sizeof(targetListSize);
+ maxsize -= sizeof(targetListSize);
+
+ /* copy each target list entry */
+ foreach(l, (List *) targetList)
+ {
+ maxsize -= sizeof(TargetEntry);
+ if (maxsize < 0)
+ elog(ERROR, "not enough space to serialize given target list");
+ srctargetEntry = (TargetEntry *) lfirst(l);
+
+ desttargetEntry = (TargetEntry *)curptr;
+ memcpy(desttargetEntry, srctargetEntry, sizeof(TargetEntry));
+
+ /*
+ * For now, lets just support for Var type of nodes.
+ *
+ * FIXME - we need to traverse target list and serialize depending
+ * on the node type.
+ */
+ desttargetEntry->expr = (Expr*) ((char*) desttargetEntry + sizeof(TargetEntry));
+ memcpy(desttargetEntry->expr, srctargetEntry->expr, sizeof(Var));
+ desttargetEntry->resname =
+ (char*) ((char*) desttargetEntry + sizeof(TargetEntry) + sizeof(Var));
+ memcpy(desttargetEntry->resname, (char*) srctargetEntry->resname,
+ strlen(srctargetEntry->resname)+1);
+
+ curptr += sizeof(TargetEntry);
+ curptr += sizeof(Var);
+ curptr += sizeof(NAMEDATALEN);
+ }
+}
+
+/*
+ * RestoreTargetList:
+ * Reads the targetlist from the specified address, restore it into given
+ * targetlist.
+ */
+void
+RestoreTargetList(List **targetList, char *start_address)
+{
+ Size targetListSize;
+ char *curptr;
+ char *colname;
+ TargetEntry *srctargetEntry;
+ TargetEntry *desttargetEntry;
+
+ memcpy(&targetListSize, start_address, sizeof(targetListSize));
+ curptr = start_address + sizeof(targetListSize);
+
+ while (targetListSize-- > 0)
+ {
+ desttargetEntry = makeNode(TargetEntry);
+ srctargetEntry = (TargetEntry *)curptr;
+ memcpy(desttargetEntry, srctargetEntry, sizeof(TargetEntry));
+
+ desttargetEntry->expr = (Expr*) copyObject((Expr*)((char*)srctargetEntry + sizeof(TargetEntry)));
+
+ /*
+ * For now, lets just support for Var type of nodes.
+ *
+ * FIXME - we need to traverse target list and restore depending
+ * on the node type.
+ */
+ colname = (char*)((char*)srctargetEntry + sizeof(TargetEntry) + sizeof(Var));
+
+ desttargetEntry->resname = colname ? pstrdup(colname) : (char*) NULL;
+
+ *targetList = lappend(*targetList, desttargetEntry);
+
+ curptr += sizeof(TargetEntry);
+ curptr += sizeof(Var);
+ curptr += sizeof(NAMEDATALEN);
+ }
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 53cfda5..131cfc5 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,22 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index c97355e..3a0583a 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 659daa2..0296323 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -106,6 +106,8 @@ int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +221,63 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * 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.
+ */
+ path->path.startup_cost = startup_cost / nWorkers;
+ path->path.total_cost = (startup_cost + run_cost) / nWorkers;
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..823abbe
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,97 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+ /*
+ * parallel scan is not supported for joins or queries containg quals.
+ *
+ * XXX - There is no reason for not to support quals, so it should be
+ * supportted in future versions of this patch.
+ */
+ if (root->simple_rel_array_size > 2 || rel->baserestrictinfo != NULL)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if(IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index bf8dbe0..6b54e1b 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -57,6 +57,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -99,6 +102,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -227,6 +233,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -342,6 +349,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -1132,6 +1146,71 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock)
+{
+ SeqScan *scan_plan;
+
+ /*
+ * Pass scan_relid as 1, this is okay for now as sequence scan worker
+ * is allowed to operate on just one relation.
+ * XXX - we should ideally get scanrelid from master backend.
+ */
+ scan_plan = make_seqscan(targetList,
+ scan_clauses,
+ 1);
+
+ scan_plan->startblock = startBlock;
+ scan_plan->endblock = endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3313,6 +3392,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index fb74d6b..49359e3 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,55 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ AclMode required_access = ACL_SELECT;
+ RangeTblEntry *rte;
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+
+ rte = makeNode(RangeTblEntry);
+ rte->rtekind = RTE_RELATION;
+ rte->relid = workerstmt->relId;
+ rte->relkind = 'r';
+ rte->requiredPerms = required_access;
+
+ scan_plan = create_worker_seqscan_plan(workerstmt->targetList, NIL,
+ workerstmt->startBlock,
+ workerstmt->endBlock);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = list_make1(rte);
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, rte->relid);;
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index e630d0b..220b92b 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 319e8b2..ce3df40 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,37 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks (refer exec_worker_message).
+ */
+ pathnode->num_blocks_per_worker = rel->pages / nWorkers;
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..3b796dd
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,579 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define SHM_PARALLEL_SCAN_QUEUE_SIZE 65536
+
+/*
+ * This structure is stored in the dynamic shared memory segment. We use
+ * it to determine whether all workers started up OK and successfully
+ * attached to their respective shared message queues.
+ */
+typedef struct
+{
+ slock_t mutex;
+ int workers_total;
+ int workers_attached;
+ int workers_ready;
+} shm_mq_header;
+
+/* Fixed-size data passed via our dynamic shared memory segment. */
+typedef struct worker_fixed_data
+{
+ Oid database_id;
+ Oid authenticated_user_id;
+ Oid current_user_id;
+ int sec_context;
+ NameData database;
+ NameData authenticated_user;
+} worker_fixed_data;
+
+/* Private state maintained by the launching backend for IPC. */
+typedef struct worker_info
+{
+ pid_t pid;
+ Oid current_user_id;
+ dsm_segment *seg;
+ BackgroundWorkerHandle *handle;
+ shm_mq_handle *responseq;
+ bool consumed;
+} worker_info;
+
+typedef struct
+{
+ int nworkers;
+ BackgroundWorkerHandle *handle[FLEXIBLE_ARRAY_MEMBER];
+} worker_state;
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PG_WORKER_MAGIC 0x50674267
+#define PG_WORKER_KEY_HDR_DATA 0
+#define PG_WORKER_KEY_FIXED_DATA 1
+#define PG_WORKER_KEY_RELID 2
+#define PG_WORKER_KEY_TARGETLIST 3
+#define PG_WORKER_KEY_BLOCKS 4
+#define PG_WORKER_FIXED_NKEYS 5
+
+void
+exec_worker_message(Datum) __attribute__((noreturn));
+
+static void
+setup_dynamic_shared_memory(Oid relId, List *targetList,
+ shm_mq_handle ***responseq,
+ dsm_segment **segp, shm_mq_header **hdrp,
+ BlockNumber numBlocksPerWorker, int nWorkers);
+static worker_state *setup_backend_workers(dsm_segment *seg, int nworkers);
+static void cleanup_background_workers(dsm_segment *seg, Datum arg);
+static void
+wait_for_workers_to_become_ready(worker_state *wstate,
+ volatile shm_mq_header *hdr);
+static bool check_worker_status(worker_state *wstate);
+static void bkworker_sigterm_handler(SIGNAL_ARGS);
+
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Oid relId, List *targetList, shm_mq_handle ***responseqp,
+ dsm_segment **segp, BlockNumber numBlocksPerWorker,
+ int nWorkers)
+{
+ shm_mq_header *hdr;
+ worker_state *wstate;
+ int i;
+
+ /* Create dynamic shared memory and table of contents. */
+ setup_dynamic_shared_memory(relId, targetList, responseqp,
+ segp, &hdr, numBlocksPerWorker, nWorkers);
+
+ /* Register backend workers. */
+ wstate = setup_backend_workers(*segp, nWorkers);
+
+ for (i = 0; i < nWorkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], wstate->handle[i]);
+
+ /* Wait for workers to become ready. */
+ wait_for_workers_to_become_ready(wstate, hdr);
+}
+
+/*
+ * Set up a dynamic shared memory segment.
+ *
+ * We set up a small control region that contains only a shm_mq_header,
+ * plus one region per message queue. There are as many message queues as
+ * the number of workers.
+ */
+static void
+setup_dynamic_shared_memory(Oid relId, List *targetList,
+ shm_mq_handle ***responseqp,
+ dsm_segment **segp, shm_mq_header **hdrp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ Size segsize, relid_len, targetlist_len;
+ dsm_segment *seg;
+ shm_toc_estimator e;
+ shm_toc *toc;
+ worker_fixed_data *fdata;
+ char *relidp;
+ char *targetlistdata;
+ int i;
+ shm_mq *mq;
+ shm_mq_header *hdr;
+ BlockNumber *num_blocks_per_worker;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /* Create dynamic shared memory and table of contents. */
+ shm_toc_initialize_estimator(&e);
+
+ shm_toc_estimate_chunk(&e, sizeof(shm_mq_header));
+
+ shm_toc_estimate_chunk(&e, sizeof(worker_fixed_data));
+
+ relid_len = EstimateScanRelationIdSpace(relId);
+ shm_toc_estimate_chunk(&e, relid_len);
+
+ targetlist_len = EstimateTargetListSpace(targetList);
+ shm_toc_estimate_chunk(&e, targetlist_len);
+
+ shm_toc_estimate_chunk(&e, sizeof(BlockNumber));
+
+ for (i = 0; i < nWorkers; ++i)
+ shm_toc_estimate_chunk(&e, (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE);
+
+ shm_toc_estimate_keys(&e, PG_WORKER_FIXED_NKEYS + nWorkers);
+
+ segsize = shm_toc_estimate(&e);
+
+ seg = dsm_create(segsize);
+ toc = shm_toc_create(PG_WORKER_MAGIC, dsm_segment_address(seg),
+ segsize);
+
+ /* Set up the header region. */
+ hdr = shm_toc_allocate(toc, sizeof(shm_mq_header));
+ SpinLockInit(&hdr->mutex);
+ hdr->workers_total = nWorkers;
+ hdr->workers_attached = 0;
+ hdr->workers_ready = 0;
+ shm_toc_insert(toc, PG_WORKER_KEY_HDR_DATA, hdr);
+
+ /* Store fixed-size data in dynamic shared memory. */
+ fdata = shm_toc_allocate(toc, sizeof(worker_fixed_data));
+ fdata->database_id = MyDatabaseId;
+ fdata->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fdata->current_user_id, &fdata->sec_context);
+ namestrcpy(&fdata->database, get_database_name(MyDatabaseId));
+ namestrcpy(&fdata->authenticated_user,
+ GetUserNameFromId(fdata->authenticated_user_id));
+ shm_toc_insert(toc, PG_WORKER_KEY_FIXED_DATA, fdata);
+
+ /* Store scan relation id in dynamic shared memory. */
+ relidp = shm_toc_allocate(toc, relid_len + 1);
+ SerializeScanRelationId(relId, relid_len, relidp);
+ shm_toc_insert(toc, PG_WORKER_KEY_RELID, relidp);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(toc, targetlist_len);
+ SerializeTargetList(targetList, targetlist_len, targetlistdata);
+ shm_toc_insert(toc, PG_WORKER_KEY_TARGETLIST, targetlistdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(toc, PG_WORKER_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Establish one message queue per worker in dynamic shared memory. */
+ for (i = 1; i <= nWorkers; ++i)
+ {
+ mq = shm_mq_create(shm_toc_allocate(toc, (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE),
+ (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE);
+ shm_toc_insert(toc, PG_WORKER_FIXED_NKEYS + i, mq);
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i-1] = shm_mq_attach(mq, seg, NULL);
+ }
+
+ /* Return results to caller. */
+ *segp = seg;
+ *hdrp = hdr;
+}
+
+/*
+ * Register backend workers.
+ */
+static worker_state *
+setup_backend_workers(dsm_segment *seg, int nWorkers)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ worker_state *wstate;
+ int i;
+
+ /*
+ * We need the worker_state object and the background worker handles to
+ * which it points to be allocated in CurTransactionContext rather than
+ * ExprContext; otherwise, they'll be destroyed before the on_dsm_detach
+ * hooks run.
+ */
+ oldcontext = MemoryContextSwitchTo(CurTransactionContext);
+
+ /* Create worker state object. */
+ wstate = MemoryContextAlloc(TopTransactionContext,
+ offsetof(worker_state, handle) +
+ sizeof(BackgroundWorkerHandle *) * nWorkers);
+ wstate->nworkers = 0;
+
+ /*
+ * Arrange to kill all the workers if we abort before or after all workers
+ * are finished hooking themselves up to the dynamic shared memory segment.
+ *
+ * XXX - For killing workers, we need to have mechanism with which it can be
+ * done before aborting the transaction.
+ */
+
+ on_dsm_detach(seg, cleanup_background_workers,
+ PointerGetDatum(wstate));
+
+ /* Configure a worker. */
+ worker.bgw_flags =
+ BGWORKER_SHMEM_ACCESS | BGWORKER_BACKEND_DATABASE_CONNECTION;
+ worker.bgw_start_time = BgWorkerStart_ConsistentState;
+ worker.bgw_restart_time = BGW_NEVER_RESTART;
+ worker.bgw_main = exec_worker_message;
+ snprintf(worker.bgw_name, BGW_MAXLEN, "backend_worker");
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(seg));
+ /* set bgw_notify_pid, so we can detect if the worker stops */
+ worker.bgw_notify_pid = MyProcPid;
+
+ /* Register the workers. */
+ for (i = 0; i < nWorkers; ++i)
+ {
+ if (!RegisterDynamicBackgroundWorker(&worker, &wstate->handle[i]))
+ ereport(ERROR,
+ (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
+ errmsg("could not register background process"),
+ errhint("You may need to increase max_worker_processes.")));
+ ++wstate->nworkers;
+ }
+
+ /* All done. */
+ MemoryContextSwitchTo(oldcontext);
+ return wstate;
+}
+
+static void
+wait_for_workers_to_become_ready(worker_state *wstate,
+ volatile shm_mq_header *hdr)
+{
+ bool save_set_latch_on_sigusr1;
+ bool result = false;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ int workers_ready;
+
+ /* If all the workers are ready, we have succeeded. */
+ SpinLockAcquire(&hdr->mutex);
+ workers_ready = hdr->workers_ready;
+ SpinLockRelease(&hdr->mutex);
+ if (workers_ready >= wstate->nworkers)
+ {
+ result = true;
+ break;
+ }
+
+ /* If any workers (or the postmaster) have died, we have failed. */
+ if (!check_worker_status(wstate))
+ {
+ result = false;
+ break;
+ }
+
+ /* Wait to be signalled. */
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+
+ /* An interrupt may have occurred while we were waiting. */
+ CHECK_FOR_INTERRUPTS();
+
+ /* Reset the latch so we don't spin. */
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ if (!result)
+ ereport(ERROR,
+ (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
+ errmsg("one or more background workers failed to start")));
+}
+
+static bool
+check_worker_status(worker_state *wstate)
+{
+ int n;
+
+ /* If any workers (or the postmaster) have died, we have failed. */
+ for (n = 0; n < wstate->nworkers; ++n)
+ {
+ BgwHandleStatus status;
+ pid_t pid;
+
+ status = GetBackgroundWorkerPid(wstate->handle[n], &pid);
+ /*if (status == BGWH_STOPPED || status == BGWH_POSTMASTER_DIED)*/
+ /*
+ * XXX - Do we need to consider BGWH_STOPPED status, if directly return
+ * false for BGWH_STOPPED, it could very well be possble that worker has
+ * exited after completing the work in which case the caller of this won't
+ * wait for other worker's status and main backend will lead to error
+ * whereas everything is normal for such a case.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ return false;
+ }
+
+ /* Otherwise, things still look OK. */
+ return true;
+}
+
+static void
+cleanup_background_workers(dsm_segment *seg, Datum arg)
+{
+ worker_state *wstate = (worker_state *) arg;
+
+ while (wstate->nworkers > 0)
+ {
+ --wstate->nworkers;
+ TerminateBackgroundWorker(wstate->handle[wstate->nworkers]);
+ }
+}
+
+
+/*
+ * exec_execute_message
+ *
+ * Process an "Execute" message for a portal
+ */
+void
+exec_worker_message(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ worker_fixed_data *fdata;
+ char *relidp;
+ char *targetlistdata;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ int myworkernumber;
+ volatile shm_mq_header *hdr;
+ Oid relId;
+ List *targetList = NIL;
+ PGPROC *registrant;
+ worker_stmt *workerstmt;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, bkworker_sigterm_handler);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "backend_worker");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "backend worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+ /*while(1)
+ {
+ }*/
+
+ /* Connect to the dynamic shared memory segment. */
+ seg = dsm_attach(DatumGetInt32(main_arg));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PG_WORKER_MAGIC, dsm_segment_address(seg));
+ if (toc == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("bad magic number in dynamic shared memory segment")));
+
+ /* Find data structures in dynamic shared memory. */
+ hdr = shm_toc_lookup(toc, PG_WORKER_KEY_HDR_DATA);
+ fdata = shm_toc_lookup(toc, PG_WORKER_KEY_FIXED_DATA);
+ relidp = shm_toc_lookup(toc, PG_WORKER_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PG_WORKER_KEY_TARGETLIST);
+ num_blocks_per_worker = shm_toc_lookup(toc, PG_WORKER_KEY_BLOCKS);
+
+ /*
+ * Acquire a worker number.
+ *
+ * Our worker number gives our identity: there may be just one
+ * worker involved in this parallel operation, or there may be many.
+ */
+ SpinLockAcquire(&hdr->mutex);
+ myworkernumber = ++hdr->workers_attached;
+ SpinLockRelease(&hdr->mutex);
+ if (myworkernumber > hdr->workers_total)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many message queue testing workers already")));
+
+ mq = shm_toc_lookup(toc, PG_WORKER_FIXED_NKEYS + myworkernumber);
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = myworkernumber * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /*
+ * Indicate that we're fully initialized and ready to begin the main part
+ * of the parallel operation.
+ *
+ * Once we signal that we're ready, the user backend is entitled to assume
+ * that our on_dsm_detach callbacks will fire before we disconnect from
+ * the shared memory segment and exit. Generally, that means we must have
+ * attached to all relevant dynamic shared memory data structures by now.
+ */
+ SpinLockAcquire(&hdr->mutex);
+ ++hdr->workers_ready;
+ SpinLockRelease(&hdr->mutex);
+ registrant = BackendPidGetProc(MyBgworkerEntry->bgw_notify_pid);
+ if (registrant == NULL)
+ {
+ elog(DEBUG1, "registrant backend has exited prematurely");
+ proc_exit(1);
+ }
+ SetLatch(®istrant->procLatch);
+
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_shm_mq(mq, responseq);
+
+ /*
+ * Initialize our user and database ID based on the strings version of
+ * the data, and then go back and check that we actually got the database
+ * and user ID that we intended to get. We do this because it's not
+ * impossible for the process that started us to die before we get here,
+ * and the user or database could be renamed in the meantime. We don't
+ * want to latch on the wrong object by accident. There should probably
+ * be a variant of BackgroundWorkerInitializeConnection that accepts OIDs
+ * rather than strings.
+ */
+ BackgroundWorkerInitializeConnection(NameStr(fdata->database),
+ NameStr(fdata->authenticated_user));
+ if (fdata->database_id != MyDatabaseId ||
+ fdata->authenticated_user_id != GetAuthenticatedUserId())
+ ereport(ERROR,
+ (errmsg("user or database renamed during backend worker startup")));
+
+ /* Restore RelationId and TargetList from main backend. */
+ RestoreScanRelationId(&relId, relidp);
+ RestoreTargetList(&targetList, targetlistdata);
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fdata->current_user_id, fdata->sec_context);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->relId = relId;
+ workerstmt->targetList = targetList;
+ workerstmt->startBlock = start_block;
+
+ /* last worker should scan all the remaining blocks. */
+ if (myworkernumber == hdr->workers_total)
+ workerstmt->endBlock = InvalidBlockNumber;
+ else
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ ProcessCompletedNotifies();
+
+ /* Signal that we are done. */
+ ReadyForQuery(DestRemote);
+
+ proc_exit(1);
+}
+
+/*
+ * When we receive a SIGTERM, we set InterruptPending and ProcDiePending just
+ * like a normal backend. The next CHECK_FOR_INTERRUPTS() will do the right
+ * thing.
+ */
+static void
+bkworker_sigterm_handler(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
\ No newline at end of file
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 6220a8e..11db15e 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -99,6 +99,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -830,6 +831,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..7de5e0e 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1132,6 +1133,105 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /* Make sure we are in a transaction command */
+ start_xact_command();
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal, NULL, 0, InvalidSnapshot);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+ (*receiver->rDestroy) (receiver);
+
+ PortalDrop(portal, false);
+
+ finish_xact_command();
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 23cbe90..69de3b8 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -619,6 +619,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2425,6 +2427,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 4a89cb7..3a6b037 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -494,6 +494,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index f2c7ca1..f88ef2e 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,7 +20,6 @@
#include "access/itup.h"
#include "access/tupdesc.h"
-
typedef struct HeapScanDescData
{
/* scan parameters */
@@ -105,4 +104,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..aa444bc
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,39 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ bool has_row_description;
+ bool complete;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41b13b2..19ec043 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,11 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1212,6 +1214,23 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ dsm_segment *seg;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index bc71fea..c48df6c 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 3e4f815..54efdc1 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -23,6 +23,7 @@
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +157,14 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Oid relId;
+ List *targetList;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 7f9eaf0..0375ce1 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 810b9c8..3a38270 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 75e2afb..a738c54 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -50,6 +50,7 @@ extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +69,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 26b17f5..901c792 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index afa5f9b..d2a2760 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 3fdc2cb..b382a27 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,9 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index 1e942c5..752bd16 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..68f2023
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,29 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Oid relId, List *targetList,
+ shm_mq_handle ***responseqp,
+ dsm_segment **segp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 60f7532..6087b5e 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index 47ff880..532d2db 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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-loadadditional_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-loadadditional_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, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Dec 5, 2014 at 8:38 PM, José Luis Tallón <jltallon@adv-solutions.net>
wrote:
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.
Also, it should consider MaxConnections configured by user.
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-loadadditional_cost = tuple estimate * cpu_tuple_cost * (l+1)/(c-1)
(for c>1, of course)
How will you identify load in above formula and what is exactly 'c'
(is it parallel workers involved?).
For now, I have managed this simply by having a configuration
variable and it seems to me that the same should be good
enough for first version, we can definitely enhance it in future
version by dynamically allocating the number of workers based
on their availability and need of query, but I think lets leave that
for another day.
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.
Idea sounds interesting and I think probably in some cases
different implementation of same node might help, but may be
at this stage if we focus on one kind of implementation (which is
a win for reasonable number of cases) and make it successful,
then doing alternative implementations will be comparatively
easier and have more chances of success.
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.
Sure, please go ahead which ever way you like to proceed.
If you want to contribute in this area/patch, then you are
welcome.
Any comments appreciated.
Thank you for your work, Amit
Many thanks to you as well for showing interest.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
Regards
David Rowley
Attachments:
merge_aggregate_state_v1.patchapplication/octet-stream; name=merge_aggregate_state_v1.patchDownload
diff --git a/doc/src/sgml/ref/create_aggregate.sgml b/doc/src/sgml/ref/create_aggregate.sgml
index eaa410b..407dc66 100644
--- a/doc/src/sgml/ref/create_aggregate.sgml
+++ b/doc/src/sgml/ref/create_aggregate.sgml
@@ -27,6 +27,8 @@ CREATE AGGREGATE <replaceable class="parameter">name</replaceable> ( [ <replacea
[ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
[ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
[ , FINALFUNC_EXTRA ]
+ [ , MERGEFUNC = <replaceable class="PARAMETER">mfunc</replaceable> ]
+ [ , MERGEFUNC_EXTRA ]
[ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
[ , MSFUNC = <replaceable class="PARAMETER">msfunc</replaceable> ]
[ , MINVFUNC = <replaceable class="PARAMETER">minvfunc</replaceable> ]
@@ -45,6 +47,8 @@ CREATE AGGREGATE <replaceable class="parameter">name</replaceable> ( [ [ <replac
[ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
[ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
[ , FINALFUNC_EXTRA ]
+ [ , MERGEFUNC = <replaceable class="PARAMETER">mfunc</replaceable> ]
+ [ , MERGEFUNC_EXTRA ]
[ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
[ , HYPOTHETICAL ]
)
@@ -58,6 +62,8 @@ CREATE AGGREGATE <replaceable class="PARAMETER">name</replaceable> (
[ , SSPACE = <replaceable class="PARAMETER">state_data_size</replaceable> ]
[ , FINALFUNC = <replaceable class="PARAMETER">ffunc</replaceable> ]
[ , FINALFUNC_EXTRA ]
+ [ , MERGEFUNC = <replaceable class="PARAMETER">mfunc</replaceable> ]
+ [ , MERGEFUNC_EXTRA ]
[ , INITCOND = <replaceable class="PARAMETER">initial_condition</replaceable> ]
[ , MSFUNC = <replaceable class="PARAMETER">msfunc</replaceable> ]
[ , MINVFUNC = <replaceable class="PARAMETER">minvfunc</replaceable> ]
diff --git a/src/backend/catalog/pg_aggregate.c b/src/backend/catalog/pg_aggregate.c
index 1ad923c..199b9bd 100644
--- a/src/backend/catalog/pg_aggregate.c
+++ b/src/backend/catalog/pg_aggregate.c
@@ -57,10 +57,12 @@ AggregateCreate(const char *aggName,
Oid variadicArgType,
List *aggtransfnName,
List *aggfinalfnName,
+ List *aggmergefnName,
List *aggmtransfnName,
List *aggminvtransfnName,
List *aggmfinalfnName,
bool finalfnExtraArgs,
+ bool mergefnExtraArgs,
bool mfinalfnExtraArgs,
List *aggsortopName,
Oid aggTransType,
@@ -77,6 +79,7 @@ AggregateCreate(const char *aggName,
Form_pg_proc proc;
Oid transfn;
Oid finalfn = InvalidOid; /* can be omitted */
+ Oid mergefn = InvalidOid; /* can be omitted */
Oid mtransfn = InvalidOid; /* can be omitted */
Oid minvtransfn = InvalidOid; /* can be omitted */
Oid mfinalfn = InvalidOid; /* can be omitted */
@@ -90,6 +93,7 @@ AggregateCreate(const char *aggName,
Oid fnArgs[FUNC_MAX_ARGS];
int nargs_transfn;
int nargs_finalfn;
+ int nargs_mergefn;
Oid procOid;
TupleDesc tupDesc;
int i;
@@ -396,6 +400,50 @@ AggregateCreate(const char *aggName,
}
Assert(OidIsValid(finaltype));
+ /* handle the mergefn, if supplied */
+ if (aggmergefnName)
+ {
+ /*
+ * If mergefnExtraArgs is specified, the transfn takes the transtype
+ * plus all args; otherwise, it just takes the transtype plus any
+ * direct args. (Non-direct args are useless at runtime, and are
+ * actually passed as NULLs, but we may need them in the function
+ * signature to allow resolution of a polymorphic agg's result type.)
+ */
+ Oid mfnVariadicArgType = variadicArgType;
+
+ /* the 1st and 2nd args must be the trans type */
+ fnArgs[0] = aggTransType;
+ fnArgs[1] = aggTransType;
+ memcpy(fnArgs + 2, aggArgTypes, numArgs * sizeof(Oid));
+ if (mergefnExtraArgs)
+ nargs_mergefn = numArgs + 2;
+ else
+ {
+ nargs_mergefn = numDirectArgs + 2;
+ if (numDirectArgs < numArgs)
+ {
+ /* variadic argument doesn't affect finalfn */
+ mfnVariadicArgType = InvalidOid;
+ }
+ }
+
+ mergefn = lookup_agg_function(aggmergefnName, nargs_mergefn,
+ fnArgs, mfnVariadicArgType,
+ &finaltype);
+
+ /*
+ * When mergefnExtraArgs is specified, the mergefn will certainly be
+ * passed at least one null argument, so complain if it's strict.
+ * Nothing bad would happen at runtime (you'd just get a null result),
+ * but it's surely not what the user wants, so let's complain now.
+ */
+ if (mergefnExtraArgs && func_strict(mergefn))
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_FUNCTION_DEFINITION),
+ errmsg("merge function with extra arguments must not be declared STRICT")));
+ }
+
/*
* If finaltype (i.e. aggregate return type) is polymorphic, inputs must
* be polymorphic also, else parser will fail to deduce result type.
@@ -423,6 +471,7 @@ AggregateCreate(const char *aggName,
errmsg("unsafe use of pseudo-type \"internal\""),
errdetail("A function returning \"internal\" must have at least one \"internal\" argument.")));
+
/*
* If a moving-aggregate implementation is supplied, look up its finalfn
* if any, and check that the implied aggregate result type matches the
diff --git a/src/backend/commands/aggregatecmds.c b/src/backend/commands/aggregatecmds.c
index fcf86dd..50d6c6e 100644
--- a/src/backend/commands/aggregatecmds.c
+++ b/src/backend/commands/aggregatecmds.c
@@ -61,10 +61,12 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
char aggKind = AGGKIND_NORMAL;
List *transfuncName = NIL;
List *finalfuncName = NIL;
+ List *mergefuncName = NIL;
List *mtransfuncName = NIL;
List *minvtransfuncName = NIL;
List *mfinalfuncName = NIL;
bool finalfuncExtraArgs = false;
+ bool mergefuncExtraArgs = false;
bool mfinalfuncExtraArgs = false;
List *sortoperatorName = NIL;
TypeName *baseType = NULL;
@@ -124,6 +126,8 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
transfuncName = defGetQualifiedName(defel);
else if (pg_strcasecmp(defel->defname, "finalfunc") == 0)
finalfuncName = defGetQualifiedName(defel);
+ else if (pg_strcasecmp(defel->defname, "mergefunc") == 0)
+ mergefuncName = defGetQualifiedName(defel);
else if (pg_strcasecmp(defel->defname, "msfunc") == 0)
mtransfuncName = defGetQualifiedName(defel);
else if (pg_strcasecmp(defel->defname, "minvfunc") == 0)
@@ -132,6 +136,8 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
mfinalfuncName = defGetQualifiedName(defel);
else if (pg_strcasecmp(defel->defname, "finalfunc_extra") == 0)
finalfuncExtraArgs = defGetBoolean(defel);
+ else if (pg_strcasecmp(defel->defname, "mergefunc_extra") == 0)
+ mergefuncExtraArgs = defGetBoolean(defel);
else if (pg_strcasecmp(defel->defname, "mfinalfunc_extra") == 0)
mfinalfuncExtraArgs = defGetBoolean(defel);
else if (pg_strcasecmp(defel->defname, "sortop") == 0)
@@ -383,10 +389,12 @@ DefineAggregate(List *name, List *args, bool oldstyle, List *parameters,
variadicArgType,
transfuncName, /* step function name */
finalfuncName, /* final function name */
+ mergefuncName, /* merge function name */
mtransfuncName, /* fwd trans function name */
minvtransfuncName, /* inv trans function name */
mfinalfuncName, /* final function name */
finalfuncExtraArgs,
+ mergefuncExtraArgs,
mfinalfuncExtraArgs,
sortoperatorName, /* sort operator name */
transTypeId, /* transition data type */
diff --git a/src/bin/pg_dump/pg_dump.c b/src/bin/pg_dump/pg_dump.c
index 4175ddc..6569a07 100644
--- a/src/bin/pg_dump/pg_dump.c
+++ b/src/bin/pg_dump/pg_dump.c
@@ -11898,10 +11898,12 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
PGresult *res;
int i_aggtransfn;
int i_aggfinalfn;
+ int i_aggmergefn;
int i_aggmtransfn;
int i_aggminvtransfn;
int i_aggmfinalfn;
int i_aggfinalextra;
+ int i_aggmergeextra;
int i_aggmfinalextra;
int i_aggsortop;
int i_hypothetical;
@@ -11914,10 +11916,12 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
int i_convertok;
const char *aggtransfn;
const char *aggfinalfn;
+ const char *aggmergefn;
const char *aggmtransfn;
const char *aggminvtransfn;
const char *aggmfinalfn;
bool aggfinalextra;
+ bool aggmergeextra;
bool aggmfinalextra;
const char *aggsortop;
char *aggsortconvop;
@@ -11944,7 +11948,26 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
selectSourceSchema(fout, agginfo->aggfn.dobj.namespace->dobj.name);
/* Get aggregate-specific details */
- if (fout->remoteVersion >= 90400)
+ if (fout->remoteVersion >= 90500) // FIXME 9.5? Maybe 10.0?
+ {
+ appendPQExpBuffer(query, "SELECT aggtransfn, "
+ "aggfinalfn, aggtranstype::pg_catalog.regtype, "
+ "aggmergefn, aggmtransfn, aggminvtransfn, "
+ "aggmfinalfn, aggmtranstype::pg_catalog.regtype, "
+ "aggfinalextra, aggmergeextra, aggmfinalextra, "
+ "aggsortop::pg_catalog.regoperator, "
+ "(aggkind = 'h') AS hypothetical, "
+ "aggtransspace, agginitval, "
+ "aggmtransspace, aggminitval, "
+ "true AS convertok, "
+ "pg_catalog.pg_get_function_arguments(p.oid) AS funcargs, "
+ "pg_catalog.pg_get_function_identity_arguments(p.oid) AS funciargs "
+ "FROM pg_catalog.pg_aggregate a, pg_catalog.pg_proc p "
+ "WHERE a.aggfnoid = p.oid "
+ "AND p.oid = '%u'::pg_catalog.oid",
+ agginfo->aggfn.dobj.catId.oid);
+ }
+ else if (fout->remoteVersion >= 90400)
{
appendPQExpBuffer(query, "SELECT aggtransfn, "
"aggfinalfn, aggtranstype::pg_catalog.regtype, "
@@ -12054,10 +12077,12 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
i_aggtransfn = PQfnumber(res, "aggtransfn");
i_aggfinalfn = PQfnumber(res, "aggfinalfn");
+ i_aggmergefn = PQfnumber(res, "aggmergefn");
i_aggmtransfn = PQfnumber(res, "aggmtransfn");
i_aggminvtransfn = PQfnumber(res, "aggminvtransfn");
i_aggmfinalfn = PQfnumber(res, "aggmfinalfn");
i_aggfinalextra = PQfnumber(res, "aggfinalextra");
+ i_aggmergeextra = PQfnumber(res, "aggmergeextra");
i_aggmfinalextra = PQfnumber(res, "aggmfinalextra");
i_aggsortop = PQfnumber(res, "aggsortop");
i_hypothetical = PQfnumber(res, "hypothetical");
@@ -12071,10 +12096,12 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
aggtransfn = PQgetvalue(res, 0, i_aggtransfn);
aggfinalfn = PQgetvalue(res, 0, i_aggfinalfn);
+ aggmergefn = PQgetvalue(res, 0, i_aggmergefn);
aggmtransfn = PQgetvalue(res, 0, i_aggmtransfn);
aggminvtransfn = PQgetvalue(res, 0, i_aggminvtransfn);
aggmfinalfn = PQgetvalue(res, 0, i_aggmfinalfn);
aggfinalextra = (PQgetvalue(res, 0, i_aggfinalextra)[0] == 't');
+ aggmergeextra = (PQgetvalue(res, 0, i_aggmergeextra)[0] == 't');
aggmfinalextra = (PQgetvalue(res, 0, i_aggmfinalextra)[0] == 't');
aggsortop = PQgetvalue(res, 0, i_aggsortop);
hypothetical = (PQgetvalue(res, 0, i_hypothetical)[0] == 't');
@@ -12159,6 +12186,14 @@ dumpAgg(Archive *fout, DumpOptions *dopt, AggInfo *agginfo)
appendPQExpBufferStr(details, ",\n FINALFUNC_EXTRA");
}
+ if (strcmp(aggmergefn, "-") != 0)
+ {
+ appendPQExpBuffer(details, ",\n MERGEFUNC = %s",
+ aggmergefn);
+ if (aggmergeextra)
+ appendPQExpBufferStr(details, ",\n MERGEFUNC_EXTRA");
+ }
+
if (strcmp(aggmtransfn, "-") != 0)
{
appendPQExpBuffer(details, ",\n MSFUNC = %s,\n MINVFUNC = %s,\n MSTYPE = %s",
diff --git a/src/include/catalog/pg_aggregate.h b/src/include/catalog/pg_aggregate.h
index 3279353..7251b7c 100644
--- a/src/include/catalog/pg_aggregate.h
+++ b/src/include/catalog/pg_aggregate.h
@@ -32,10 +32,12 @@
* aggnumdirectargs number of arguments that are "direct" arguments
* aggtransfn transition function
* aggfinalfn final function (0 if none)
+ * aggmergefn merge function (0 if none)
* aggmtransfn forward function for moving-aggregate mode (0 if none)
* aggminvtransfn inverse function for moving-aggregate mode (0 if none)
* aggmfinalfn final function for moving-aggregate mode (0 if none)
* aggfinalextra true to pass extra dummy arguments to aggfinalfn
+ * aggmergeextra true to pass extra dummy arguments to aggmergefn
* aggmfinalextra true to pass extra dummy arguments to aggmfinalfn
* aggsortop associated sort operator (0 if none)
* aggtranstype type of aggregate's transition (state) data
@@ -55,10 +57,12 @@ CATALOG(pg_aggregate,2600) BKI_WITHOUT_OIDS
int16 aggnumdirectargs;
regproc aggtransfn;
regproc aggfinalfn;
+ regproc aggmergefn;
regproc aggmtransfn;
regproc aggminvtransfn;
regproc aggmfinalfn;
bool aggfinalextra;
+ bool aggmergeextra;
bool aggmfinalextra;
Oid aggsortop;
Oid aggtranstype;
@@ -84,24 +88,26 @@ typedef FormData_pg_aggregate *Form_pg_aggregate;
* ----------------
*/
-#define Natts_pg_aggregate 17
+#define Natts_pg_aggregate 19
#define Anum_pg_aggregate_aggfnoid 1
#define Anum_pg_aggregate_aggkind 2
#define Anum_pg_aggregate_aggnumdirectargs 3
#define Anum_pg_aggregate_aggtransfn 4
#define Anum_pg_aggregate_aggfinalfn 5
-#define Anum_pg_aggregate_aggmtransfn 6
-#define Anum_pg_aggregate_aggminvtransfn 7
-#define Anum_pg_aggregate_aggmfinalfn 8
-#define Anum_pg_aggregate_aggfinalextra 9
-#define Anum_pg_aggregate_aggmfinalextra 10
-#define Anum_pg_aggregate_aggsortop 11
-#define Anum_pg_aggregate_aggtranstype 12
-#define Anum_pg_aggregate_aggtransspace 13
-#define Anum_pg_aggregate_aggmtranstype 14
-#define Anum_pg_aggregate_aggmtransspace 15
-#define Anum_pg_aggregate_agginitval 16
-#define Anum_pg_aggregate_aggminitval 17
+#define Anum_pg_aggregate_aggmergefn 6
+#define Anum_pg_aggregate_aggmtransfn 7
+#define Anum_pg_aggregate_aggminvtransfn 8
+#define Anum_pg_aggregate_aggmfinalfn 9
+#define Anum_pg_aggregate_aggfinalextra 10
+#define Anum_pg_aggregate_aggmergeextra 11
+#define Anum_pg_aggregate_aggmfinalextra 12
+#define Anum_pg_aggregate_aggsortop 13
+#define Anum_pg_aggregate_aggtranstype 14
+#define Anum_pg_aggregate_aggtransspace 15
+#define Anum_pg_aggregate_aggmtranstype 16
+#define Anum_pg_aggregate_aggmtransspace 17
+#define Anum_pg_aggregate_agginitval 18
+#define Anum_pg_aggregate_aggminitval 19
/*
* Symbolic values for aggkind column. We distinguish normal aggregates
@@ -125,180 +131,180 @@ typedef FormData_pg_aggregate *Form_pg_aggregate;
*/
/* avg */
-DATA(insert ( 2100 n 0 int8_avg_accum numeric_avg int8_avg_accum int8_accum_inv numeric_avg f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2101 n 0 int4_avg_accum int8_avg int4_avg_accum int4_avg_accum_inv int8_avg f f 0 1016 0 1016 0 "{0,0}" "{0,0}" ));
-DATA(insert ( 2102 n 0 int2_avg_accum int8_avg int2_avg_accum int2_avg_accum_inv int8_avg f f 0 1016 0 1016 0 "{0,0}" "{0,0}" ));
-DATA(insert ( 2103 n 0 numeric_avg_accum numeric_avg numeric_avg_accum numeric_accum_inv numeric_avg f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2104 n 0 float4_accum float8_avg - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2105 n 0 float8_accum float8_avg - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2106 n 0 interval_accum interval_avg interval_accum interval_accum_inv interval_avg f f 0 1187 0 1187 0 "{0 second,0 second}" "{0 second,0 second}" ));
+DATA(insert ( 2100 n 0 int8_avg_accum numeric_avg - int8_avg_accum int8_accum_inv numeric_avg f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2101 n 0 int4_avg_accum int8_avg - int4_avg_accum int4_avg_accum_inv int8_avg f f f 0 1016 0 1016 0 "{0,0}" "{0,0}" ));
+DATA(insert ( 2102 n 0 int2_avg_accum int8_avg - int2_avg_accum int2_avg_accum_inv int8_avg f f f 0 1016 0 1016 0 "{0,0}" "{0,0}" ));
+DATA(insert ( 2103 n 0 numeric_avg_accum numeric_avg - numeric_avg_accum numeric_accum_inv numeric_avg f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2104 n 0 float4_accum float8_avg - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2105 n 0 float8_accum float8_avg - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2106 n 0 interval_accum interval_avg - interval_accum interval_accum_inv interval_avg f f f 0 1187 0 1187 0 "{0 second,0 second}" "{0 second,0 second}" ));
/* sum */
-DATA(insert ( 2107 n 0 int8_avg_accum numeric_sum int8_avg_accum int8_accum_inv numeric_sum f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2108 n 0 int4_sum - int4_avg_accum int4_avg_accum_inv int2int4_sum f f 0 20 0 1016 0 _null_ "{0,0}" ));
-DATA(insert ( 2109 n 0 int2_sum - int2_avg_accum int2_avg_accum_inv int2int4_sum f f 0 20 0 1016 0 _null_ "{0,0}" ));
-DATA(insert ( 2110 n 0 float4pl - - - - f f 0 700 0 0 0 _null_ _null_ ));
-DATA(insert ( 2111 n 0 float8pl - - - - f f 0 701 0 0 0 _null_ _null_ ));
-DATA(insert ( 2112 n 0 cash_pl - cash_pl cash_mi - f f 0 790 0 790 0 _null_ _null_ ));
-DATA(insert ( 2113 n 0 interval_pl - interval_pl interval_mi - f f 0 1186 0 1186 0 _null_ _null_ ));
-DATA(insert ( 2114 n 0 numeric_avg_accum numeric_sum numeric_avg_accum numeric_accum_inv numeric_sum f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2107 n 0 int8_avg_accum numeric_sum - int8_avg_accum int8_accum_inv numeric_sum f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2108 n 0 int4_sum - - int4_avg_accum int4_avg_accum_inv int2int4_sum f f f 0 20 0 1016 0 _null_ "{0,0}" ));
+DATA(insert ( 2109 n 0 int2_sum - - int2_avg_accum int2_avg_accum_inv int2int4_sum f f f 0 20 0 1016 0 _null_ "{0,0}" ));
+DATA(insert ( 2110 n 0 float4pl - float4pl - - - f f f 0 700 0 0 0 _null_ _null_ ));
+DATA(insert ( 2111 n 0 float8pl - float8pl - - - f f f 0 701 0 0 0 _null_ _null_ ));
+DATA(insert ( 2112 n 0 cash_pl - cash_pl cash_pl cash_mi - f f f 0 790 0 790 0 _null_ _null_ ));
+DATA(insert ( 2113 n 0 interval_pl - interval_pl interval_pl interval_mi - f f f 0 1186 0 1186 0 _null_ _null_ ));
+DATA(insert ( 2114 n 0 numeric_avg_accum numeric_sum - numeric_avg_accum numeric_accum_inv numeric_sum f f f 0 2281 128 2281 128 _null_ _null_ ));
/* max */
-DATA(insert ( 2115 n 0 int8larger - - - - f f 413 20 0 0 0 _null_ _null_ ));
-DATA(insert ( 2116 n 0 int4larger - - - - f f 521 23 0 0 0 _null_ _null_ ));
-DATA(insert ( 2117 n 0 int2larger - - - - f f 520 21 0 0 0 _null_ _null_ ));
-DATA(insert ( 2118 n 0 oidlarger - - - - f f 610 26 0 0 0 _null_ _null_ ));
-DATA(insert ( 2119 n 0 float4larger - - - - f f 623 700 0 0 0 _null_ _null_ ));
-DATA(insert ( 2120 n 0 float8larger - - - - f f 674 701 0 0 0 _null_ _null_ ));
-DATA(insert ( 2121 n 0 int4larger - - - - f f 563 702 0 0 0 _null_ _null_ ));
-DATA(insert ( 2122 n 0 date_larger - - - - f f 1097 1082 0 0 0 _null_ _null_ ));
-DATA(insert ( 2123 n 0 time_larger - - - - f f 1112 1083 0 0 0 _null_ _null_ ));
-DATA(insert ( 2124 n 0 timetz_larger - - - - f f 1554 1266 0 0 0 _null_ _null_ ));
-DATA(insert ( 2125 n 0 cashlarger - - - - f f 903 790 0 0 0 _null_ _null_ ));
-DATA(insert ( 2126 n 0 timestamp_larger - - - - f f 2064 1114 0 0 0 _null_ _null_ ));
-DATA(insert ( 2127 n 0 timestamptz_larger - - - - f f 1324 1184 0 0 0 _null_ _null_ ));
-DATA(insert ( 2128 n 0 interval_larger - - - - f f 1334 1186 0 0 0 _null_ _null_ ));
-DATA(insert ( 2129 n 0 text_larger - - - - f f 666 25 0 0 0 _null_ _null_ ));
-DATA(insert ( 2130 n 0 numeric_larger - - - - f f 1756 1700 0 0 0 _null_ _null_ ));
-DATA(insert ( 2050 n 0 array_larger - - - - f f 1073 2277 0 0 0 _null_ _null_ ));
-DATA(insert ( 2244 n 0 bpchar_larger - - - - f f 1060 1042 0 0 0 _null_ _null_ ));
-DATA(insert ( 2797 n 0 tidlarger - - - - f f 2800 27 0 0 0 _null_ _null_ ));
-DATA(insert ( 3526 n 0 enum_larger - - - - f f 3519 3500 0 0 0 _null_ _null_ ));
-DATA(insert ( 3564 n 0 network_larger - - - - f f 1205 869 0 0 0 _null_ _null_ ));
+DATA(insert ( 2115 n 0 int8larger - int8larger - - - f f f 413 20 0 0 0 _null_ _null_ ));
+DATA(insert ( 2116 n 0 int4larger - int4larger - - - f f f 521 23 0 0 0 _null_ _null_ ));
+DATA(insert ( 2117 n 0 int2larger - int2larger - - - f f f 520 21 0 0 0 _null_ _null_ ));
+DATA(insert ( 2118 n 0 oidlarger - oidlarger - - - f f f 610 26 0 0 0 _null_ _null_ ));
+DATA(insert ( 2119 n 0 float4larger - float4larger - - - f f f 623 700 0 0 0 _null_ _null_ ));
+DATA(insert ( 2120 n 0 float8larger - float8larger - - - f f f 674 701 0 0 0 _null_ _null_ ));
+DATA(insert ( 2121 n 0 int4larger - int4larger - - - f f f 563 702 0 0 0 _null_ _null_ ));
+DATA(insert ( 2122 n 0 date_larger - date_larger - - - f f f 1097 1082 0 0 0 _null_ _null_ ));
+DATA(insert ( 2123 n 0 time_larger - time_larger - - - f f f 1112 1083 0 0 0 _null_ _null_ ));
+DATA(insert ( 2124 n 0 timetz_larger - timetz_larger - - - f f f 1554 1266 0 0 0 _null_ _null_ ));
+DATA(insert ( 2125 n 0 cashlarger - cashlarger - - - f f f 903 790 0 0 0 _null_ _null_ ));
+DATA(insert ( 2126 n 0 timestamp_larger - timestamp_larger - - - f f f 2064 1114 0 0 0 _null_ _null_ ));
+DATA(insert ( 2127 n 0 timestamptz_larger - timestamptz_larger - - - f f f 1324 1184 0 0 0 _null_ _null_ ));
+DATA(insert ( 2128 n 0 interval_larger - interval_larger - - - f f f 1334 1186 0 0 0 _null_ _null_ ));
+DATA(insert ( 2129 n 0 text_larger - text_larger - - - f f f 666 25 0 0 0 _null_ _null_ ));
+DATA(insert ( 2130 n 0 numeric_larger - numeric_larger - - - f f f 1756 1700 0 0 0 _null_ _null_ ));
+DATA(insert ( 2050 n 0 array_larger - array_larger - - - f f f 1073 2277 0 0 0 _null_ _null_ ));
+DATA(insert ( 2244 n 0 bpchar_larger - bpchar_larger - - - f f f 1060 1042 0 0 0 _null_ _null_ ));
+DATA(insert ( 2797 n 0 tidlarger - tidlarger - - - f f f 2800 27 0 0 0 _null_ _null_ ));
+DATA(insert ( 3526 n 0 enum_larger - enum_larger - - - f f f 3519 3500 0 0 0 _null_ _null_ ));
+DATA(insert ( 3564 n 0 network_larger - network_larger - - - f f f 1205 869 0 0 0 _null_ _null_ ));
/* min */
-DATA(insert ( 2131 n 0 int8smaller - - - - f f 412 20 0 0 0 _null_ _null_ ));
-DATA(insert ( 2132 n 0 int4smaller - - - - f f 97 23 0 0 0 _null_ _null_ ));
-DATA(insert ( 2133 n 0 int2smaller - - - - f f 95 21 0 0 0 _null_ _null_ ));
-DATA(insert ( 2134 n 0 oidsmaller - - - - f f 609 26 0 0 0 _null_ _null_ ));
-DATA(insert ( 2135 n 0 float4smaller - - - - f f 622 700 0 0 0 _null_ _null_ ));
-DATA(insert ( 2136 n 0 float8smaller - - - - f f 672 701 0 0 0 _null_ _null_ ));
-DATA(insert ( 2137 n 0 int4smaller - - - - f f 562 702 0 0 0 _null_ _null_ ));
-DATA(insert ( 2138 n 0 date_smaller - - - - f f 1095 1082 0 0 0 _null_ _null_ ));
-DATA(insert ( 2139 n 0 time_smaller - - - - f f 1110 1083 0 0 0 _null_ _null_ ));
-DATA(insert ( 2140 n 0 timetz_smaller - - - - f f 1552 1266 0 0 0 _null_ _null_ ));
-DATA(insert ( 2141 n 0 cashsmaller - - - - f f 902 790 0 0 0 _null_ _null_ ));
-DATA(insert ( 2142 n 0 timestamp_smaller - - - - f f 2062 1114 0 0 0 _null_ _null_ ));
-DATA(insert ( 2143 n 0 timestamptz_smaller - - - - f f 1322 1184 0 0 0 _null_ _null_ ));
-DATA(insert ( 2144 n 0 interval_smaller - - - - f f 1332 1186 0 0 0 _null_ _null_ ));
-DATA(insert ( 2145 n 0 text_smaller - - - - f f 664 25 0 0 0 _null_ _null_ ));
-DATA(insert ( 2146 n 0 numeric_smaller - - - - f f 1754 1700 0 0 0 _null_ _null_ ));
-DATA(insert ( 2051 n 0 array_smaller - - - - f f 1072 2277 0 0 0 _null_ _null_ ));
-DATA(insert ( 2245 n 0 bpchar_smaller - - - - f f 1058 1042 0 0 0 _null_ _null_ ));
-DATA(insert ( 2798 n 0 tidsmaller - - - - f f 2799 27 0 0 0 _null_ _null_ ));
-DATA(insert ( 3527 n 0 enum_smaller - - - - f f 3518 3500 0 0 0 _null_ _null_ ));
-DATA(insert ( 3565 n 0 network_smaller - - - - f f 1203 869 0 0 0 _null_ _null_ ));
+DATA(insert ( 2131 n 0 int8smaller - int8smaller - - - f f f 412 20 0 0 0 _null_ _null_ ));
+DATA(insert ( 2132 n 0 int4smaller - int4smaller - - - f f f 97 23 0 0 0 _null_ _null_ ));
+DATA(insert ( 2133 n 0 int2smaller - int2smaller - - - f f f 95 21 0 0 0 _null_ _null_ ));
+DATA(insert ( 2134 n 0 oidsmaller - oidsmaller - - - f f f 609 26 0 0 0 _null_ _null_ ));
+DATA(insert ( 2135 n 0 float4smaller - float4smaller - - - f f f 622 700 0 0 0 _null_ _null_ ));
+DATA(insert ( 2136 n 0 float8smaller - float8smaller - - - f f f 672 701 0 0 0 _null_ _null_ ));
+DATA(insert ( 2137 n 0 int4smaller - int4smaller - - - f f f 562 702 0 0 0 _null_ _null_ ));
+DATA(insert ( 2138 n 0 date_smaller - date_smaller - - - f f f 1095 1082 0 0 0 _null_ _null_ ));
+DATA(insert ( 2139 n 0 time_smaller - time_smaller - - - f f f 1110 1083 0 0 0 _null_ _null_ ));
+DATA(insert ( 2140 n 0 timetz_smaller - timetz_smaller - - - f f f 1552 1266 0 0 0 _null_ _null_ ));
+DATA(insert ( 2141 n 0 cashsmaller - cashsmaller - - - f f f 902 790 0 0 0 _null_ _null_ ));
+DATA(insert ( 2142 n 0 timestamp_smaller - timestamp_smaller - - - f f f 2062 1114 0 0 0 _null_ _null_ ));
+DATA(insert ( 2143 n 0 timestamptz_smaller - timestamptz_smaller - - - f f f 1322 1184 0 0 0 _null_ _null_ ));
+DATA(insert ( 2144 n 0 interval_smaller - interval_smaller - - - f f f 1332 1186 0 0 0 _null_ _null_ ));
+DATA(insert ( 2145 n 0 text_smaller - text_smaller - - - f f f 664 25 0 0 0 _null_ _null_ ));
+DATA(insert ( 2146 n 0 numeric_smaller - numeric_smaller - - - f f f 1754 1700 0 0 0 _null_ _null_ ));
+DATA(insert ( 2051 n 0 array_smaller - array_smaller - - - f f f 1072 2277 0 0 0 _null_ _null_ ));
+DATA(insert ( 2245 n 0 bpchar_smaller - bpchar_smaller - - - f f f 1058 1042 0 0 0 _null_ _null_ ));
+DATA(insert ( 2798 n 0 tidsmaller - tidsmaller - - - f f f 2799 27 0 0 0 _null_ _null_ ));
+DATA(insert ( 3527 n 0 enum_smaller - enum_smaller - - - f f f 3518 3500 0 0 0 _null_ _null_ ));
+DATA(insert ( 3565 n 0 network_smaller - network_smaller - - - f f f 1203 869 0 0 0 _null_ _null_ ));
/* count */
-DATA(insert ( 2147 n 0 int8inc_any - int8inc_any int8dec_any - f f 0 20 0 20 0 "0" "0" ));
-DATA(insert ( 2803 n 0 int8inc - int8inc int8dec - f f 0 20 0 20 0 "0" "0" ));
+DATA(insert ( 2147 n 0 int8inc_any - - int8inc_any int8dec_any - f f f 0 20 0 20 0 "0" "0" ));
+DATA(insert ( 2803 n 0 int8inc - - int8inc int8dec - f f f 0 20 0 20 0 "0" "0" ));
/* var_pop */
-DATA(insert ( 2718 n 0 int8_accum numeric_var_pop int8_accum int8_accum_inv numeric_var_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2719 n 0 int4_accum numeric_var_pop int4_accum int4_accum_inv numeric_var_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2720 n 0 int2_accum numeric_var_pop int2_accum int2_accum_inv numeric_var_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2721 n 0 float4_accum float8_var_pop - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2722 n 0 float8_accum float8_var_pop - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2723 n 0 numeric_accum numeric_var_pop numeric_accum numeric_accum_inv numeric_var_pop f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2718 n 0 int8_accum numeric_var_pop - int8_accum int8_accum_inv numeric_var_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2719 n 0 int4_accum numeric_var_pop - int4_accum int4_accum_inv numeric_var_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2720 n 0 int2_accum numeric_var_pop - int2_accum int2_accum_inv numeric_var_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2721 n 0 float4_accum float8_var_pop - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2722 n 0 float8_accum float8_var_pop - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2723 n 0 numeric_accum numeric_var_pop - numeric_accum numeric_accum_inv numeric_var_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
/* var_samp */
-DATA(insert ( 2641 n 0 int8_accum numeric_var_samp int8_accum int8_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2642 n 0 int4_accum numeric_var_samp int4_accum int4_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2643 n 0 int2_accum numeric_var_samp int2_accum int2_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2644 n 0 float4_accum float8_var_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2645 n 0 float8_accum float8_var_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2646 n 0 numeric_accum numeric_var_samp numeric_accum numeric_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2641 n 0 int8_accum numeric_var_samp - int8_accum int8_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2642 n 0 int4_accum numeric_var_samp - int4_accum int4_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2643 n 0 int2_accum numeric_var_samp - int2_accum int2_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2644 n 0 float4_accum float8_var_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2645 n 0 float8_accum float8_var_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2646 n 0 numeric_accum numeric_var_samp - numeric_accum numeric_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
/* variance: historical Postgres syntax for var_samp */
-DATA(insert ( 2148 n 0 int8_accum numeric_var_samp int8_accum int8_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2149 n 0 int4_accum numeric_var_samp int4_accum int4_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2150 n 0 int2_accum numeric_var_samp int2_accum int2_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2151 n 0 float4_accum float8_var_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2152 n 0 float8_accum float8_var_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2153 n 0 numeric_accum numeric_var_samp numeric_accum numeric_accum_inv numeric_var_samp f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2148 n 0 int8_accum numeric_var_samp - int8_accum int8_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2149 n 0 int4_accum numeric_var_samp - int4_accum int4_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2150 n 0 int2_accum numeric_var_samp - int2_accum int2_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2151 n 0 float4_accum float8_var_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2152 n 0 float8_accum float8_var_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2153 n 0 numeric_accum numeric_var_samp - numeric_accum numeric_accum_inv numeric_var_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
/* stddev_pop */
-DATA(insert ( 2724 n 0 int8_accum numeric_stddev_pop int8_accum int8_accum_inv numeric_stddev_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2725 n 0 int4_accum numeric_stddev_pop int4_accum int4_accum_inv numeric_stddev_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2726 n 0 int2_accum numeric_stddev_pop int2_accum int2_accum_inv numeric_stddev_pop f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2727 n 0 float4_accum float8_stddev_pop - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2728 n 0 float8_accum float8_stddev_pop - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2729 n 0 numeric_accum numeric_stddev_pop numeric_accum numeric_accum_inv numeric_stddev_pop f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2724 n 0 int8_accum numeric_stddev_pop - int8_accum int8_accum_inv numeric_stddev_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2725 n 0 int4_accum numeric_stddev_pop - int4_accum int4_accum_inv numeric_stddev_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2726 n 0 int2_accum numeric_stddev_pop - int2_accum int2_accum_inv numeric_stddev_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2727 n 0 float4_accum float8_stddev_pop - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2728 n 0 float8_accum float8_stddev_pop - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2729 n 0 numeric_accum numeric_stddev_pop - numeric_accum numeric_accum_inv numeric_stddev_pop f f f 0 2281 128 2281 128 _null_ _null_ ));
/* stddev_samp */
-DATA(insert ( 2712 n 0 int8_accum numeric_stddev_samp int8_accum int8_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2713 n 0 int4_accum numeric_stddev_samp int4_accum int4_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2714 n 0 int2_accum numeric_stddev_samp int2_accum int2_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2715 n 0 float4_accum float8_stddev_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2716 n 0 float8_accum float8_stddev_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2717 n 0 numeric_accum numeric_stddev_samp numeric_accum numeric_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2712 n 0 int8_accum numeric_stddev_samp - int8_accum int8_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2713 n 0 int4_accum numeric_stddev_samp - int4_accum int4_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2714 n 0 int2_accum numeric_stddev_samp - int2_accum int2_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2715 n 0 float4_accum float8_stddev_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2716 n 0 float8_accum float8_stddev_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2717 n 0 numeric_accum numeric_stddev_samp - numeric_accum numeric_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
/* stddev: historical Postgres syntax for stddev_samp */
-DATA(insert ( 2154 n 0 int8_accum numeric_stddev_samp int8_accum int8_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2155 n 0 int4_accum numeric_stddev_samp int4_accum int4_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2156 n 0 int2_accum numeric_stddev_samp int2_accum int2_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
-DATA(insert ( 2157 n 0 float4_accum float8_stddev_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2158 n 0 float8_accum float8_stddev_samp - - - f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
-DATA(insert ( 2159 n 0 numeric_accum numeric_stddev_samp numeric_accum numeric_accum_inv numeric_stddev_samp f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2154 n 0 int8_accum numeric_stddev_samp - int8_accum int8_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2155 n 0 int4_accum numeric_stddev_samp - int4_accum int4_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2156 n 0 int2_accum numeric_stddev_samp - int2_accum int2_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
+DATA(insert ( 2157 n 0 float4_accum float8_stddev_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2158 n 0 float8_accum float8_stddev_samp - - - - f f f 0 1022 0 0 0 "{0,0,0}" _null_ ));
+DATA(insert ( 2159 n 0 numeric_accum numeric_stddev_samp - numeric_accum numeric_accum_inv numeric_stddev_samp f f f 0 2281 128 2281 128 _null_ _null_ ));
/* SQL2003 binary regression aggregates */
-DATA(insert ( 2818 n 0 int8inc_float8_float8 - - - - f f 0 20 0 0 0 "0" _null_ ));
-DATA(insert ( 2819 n 0 float8_regr_accum float8_regr_sxx - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2820 n 0 float8_regr_accum float8_regr_syy - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2821 n 0 float8_regr_accum float8_regr_sxy - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2822 n 0 float8_regr_accum float8_regr_avgx - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2823 n 0 float8_regr_accum float8_regr_avgy - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2824 n 0 float8_regr_accum float8_regr_r2 - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2825 n 0 float8_regr_accum float8_regr_slope - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2826 n 0 float8_regr_accum float8_regr_intercept - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2827 n 0 float8_regr_accum float8_covar_pop - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2828 n 0 float8_regr_accum float8_covar_samp - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
-DATA(insert ( 2829 n 0 float8_regr_accum float8_corr - - - f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2818 n 0 int8inc_float8_float8 - - - - - f f f 0 20 0 0 0 "0" _null_ ));
+DATA(insert ( 2819 n 0 float8_regr_accum float8_regr_sxx - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2820 n 0 float8_regr_accum float8_regr_syy - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2821 n 0 float8_regr_accum float8_regr_sxy - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2822 n 0 float8_regr_accum float8_regr_avgx - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2823 n 0 float8_regr_accum float8_regr_avgy - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2824 n 0 float8_regr_accum float8_regr_r2 - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2825 n 0 float8_regr_accum float8_regr_slope - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2826 n 0 float8_regr_accum float8_regr_intercept - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2827 n 0 float8_regr_accum float8_covar_pop - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2828 n 0 float8_regr_accum float8_covar_samp - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
+DATA(insert ( 2829 n 0 float8_regr_accum float8_corr - - - - f f f 0 1022 0 0 0 "{0,0,0,0,0,0}" _null_ ));
/* boolean-and and boolean-or */
-DATA(insert ( 2517 n 0 booland_statefunc - bool_accum bool_accum_inv bool_alltrue f f 58 16 0 2281 16 _null_ _null_ ));
-DATA(insert ( 2518 n 0 boolor_statefunc - bool_accum bool_accum_inv bool_anytrue f f 59 16 0 2281 16 _null_ _null_ ));
-DATA(insert ( 2519 n 0 booland_statefunc - bool_accum bool_accum_inv bool_alltrue f f 58 16 0 2281 16 _null_ _null_ ));
+DATA(insert ( 2517 n 0 booland_statefunc - - bool_accum bool_accum_inv bool_alltrue f f f 58 16 0 2281 16 _null_ _null_ ));
+DATA(insert ( 2518 n 0 boolor_statefunc - - bool_accum bool_accum_inv bool_anytrue f f f 59 16 0 2281 16 _null_ _null_ ));
+DATA(insert ( 2519 n 0 booland_statefunc - - bool_accum bool_accum_inv bool_alltrue f f f 58 16 0 2281 16 _null_ _null_ ));
/* bitwise integer */
-DATA(insert ( 2236 n 0 int2and - - - - f f 0 21 0 0 0 _null_ _null_ ));
-DATA(insert ( 2237 n 0 int2or - - - - f f 0 21 0 0 0 _null_ _null_ ));
-DATA(insert ( 2238 n 0 int4and - - - - f f 0 23 0 0 0 _null_ _null_ ));
-DATA(insert ( 2239 n 0 int4or - - - - f f 0 23 0 0 0 _null_ _null_ ));
-DATA(insert ( 2240 n 0 int8and - - - - f f 0 20 0 0 0 _null_ _null_ ));
-DATA(insert ( 2241 n 0 int8or - - - - f f 0 20 0 0 0 _null_ _null_ ));
-DATA(insert ( 2242 n 0 bitand - - - - f f 0 1560 0 0 0 _null_ _null_ ));
-DATA(insert ( 2243 n 0 bitor - - - - f f 0 1560 0 0 0 _null_ _null_ ));
+DATA(insert ( 2236 n 0 int2and - int2and - - - f f f 0 21 0 0 0 _null_ _null_ ));
+DATA(insert ( 2237 n 0 int2or - int2or - - - f f f 0 21 0 0 0 _null_ _null_ ));
+DATA(insert ( 2238 n 0 int4and - int4and - - - f f f 0 23 0 0 0 _null_ _null_ ));
+DATA(insert ( 2239 n 0 int4or - int4or - - - f f f 0 23 0 0 0 _null_ _null_ ));
+DATA(insert ( 2240 n 0 int8and - int8and - - - f f f 0 20 0 0 0 _null_ _null_ ));
+DATA(insert ( 2241 n 0 int8or - int8or - - - f f f 0 20 0 0 0 _null_ _null_ ));
+DATA(insert ( 2242 n 0 bitand - bitand - - - f f f 0 1560 0 0 0 _null_ _null_ ));
+DATA(insert ( 2243 n 0 bitor - bitor - - - f f f 0 1560 0 0 0 _null_ _null_ ));
/* xml */
-DATA(insert ( 2901 n 0 xmlconcat2 - - - - f f 0 142 0 0 0 _null_ _null_ ));
+DATA(insert ( 2901 n 0 xmlconcat2 - - - - - f f f 0 142 0 0 0 _null_ _null_ ));
/* array */
-DATA(insert ( 2335 n 0 array_agg_transfn array_agg_finalfn - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 4053 n 0 array_agg_array_transfn array_agg_array_finalfn - - - t f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 2335 n 0 array_agg_transfn array_agg_finalfn - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 4053 n 0 array_agg_array_transfn array_agg_array_finalfn - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
/* text */
-DATA(insert ( 3538 n 0 string_agg_transfn string_agg_finalfn - - - f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3538 n 0 string_agg_transfn string_agg_finalfn - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
/* bytea */
-DATA(insert ( 3545 n 0 bytea_string_agg_transfn bytea_string_agg_finalfn - - - f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3545 n 0 bytea_string_agg_transfn bytea_string_agg_finalfn - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
/* json */
-DATA(insert ( 3175 n 0 json_agg_transfn json_agg_finalfn - - - f f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3197 n 0 json_object_agg_transfn json_object_agg_finalfn - - - f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3175 n 0 json_agg_transfn json_agg_finalfn - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3197 n 0 json_object_agg_transfn json_object_agg_finalfn - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
/* ordered-set and hypothetical-set aggregates */
-DATA(insert ( 3972 o 1 ordered_set_transition percentile_disc_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3974 o 1 ordered_set_transition percentile_cont_float8_final - - - f f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3976 o 1 ordered_set_transition percentile_cont_interval_final - - - f f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3978 o 1 ordered_set_transition percentile_disc_multi_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3980 o 1 ordered_set_transition percentile_cont_float8_multi_final - - - f f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3982 o 1 ordered_set_transition percentile_cont_interval_multi_final - - - f f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3984 o 0 ordered_set_transition mode_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3986 h 1 ordered_set_transition_multi rank_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3988 h 1 ordered_set_transition_multi percent_rank_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3990 h 1 ordered_set_transition_multi cume_dist_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
-DATA(insert ( 3992 h 1 ordered_set_transition_multi dense_rank_final - - - t f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3972 o 1 ordered_set_transition percentile_disc_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3974 o 1 ordered_set_transition percentile_cont_float8_final - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3976 o 1 ordered_set_transition percentile_cont_interval_final - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3978 o 1 ordered_set_transition percentile_disc_multi_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3980 o 1 ordered_set_transition percentile_cont_float8_multi_final - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3982 o 1 ordered_set_transition percentile_cont_interval_multi_final - - - - f f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3984 o 0 ordered_set_transition mode_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3986 h 1 ordered_set_transition_multi rank_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3988 h 1 ordered_set_transition_multi percent_rank_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3990 h 1 ordered_set_transition_multi cume_dist_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
+DATA(insert ( 3992 h 1 ordered_set_transition_multi dense_rank_final - - - - t f f 0 2281 0 0 0 _null_ _null_ ));
/*
@@ -317,10 +323,12 @@ extern Oid AggregateCreate(const char *aggName,
Oid variadicArgType,
List *aggtransfnName,
List *aggfinalfnName,
+ List *aggmergefnName,
List *aggmtransfnName,
List *aggminvtransfnName,
List *aggmfinalfnName,
bool finalfnExtraArgs,
+ bool mergefnExtraArgs,
bool mfinalfnExtraArgs,
List *aggsortopName,
Oid aggTransType,
diff --git a/src/test/regress/expected/create_aggregate.out b/src/test/regress/expected/create_aggregate.out
index 82a34fb..3446b00 100644
--- a/src/test/regress/expected/create_aggregate.out
+++ b/src/test/regress/expected/create_aggregate.out
@@ -101,6 +101,13 @@ CREATE AGGREGATE sumdouble (float8)
msfunc = float8pl,
minvfunc = float8mi
);
+-- aggregate merge functions
+CREATE AGGREGATE mymax (int)
+(
+ stype = int4,
+ sfunc = int4larger,
+ mergefunc = int4larger
+);
-- invalid: nonstrict inverse with strict forward function
CREATE FUNCTION float8mi_n(float8, float8) RETURNS float8 AS
$$ SELECT $1 - $2; $$
diff --git a/src/test/regress/sql/create_aggregate.sql b/src/test/regress/sql/create_aggregate.sql
index 0ec1572..1c18ffd 100644
--- a/src/test/regress/sql/create_aggregate.sql
+++ b/src/test/regress/sql/create_aggregate.sql
@@ -115,6 +115,14 @@ CREATE AGGREGATE sumdouble (float8)
minvfunc = float8mi
);
+-- aggregate merge functions
+CREATE AGGREGATE mymax (int)
+(
+ stype = int4,
+ sfunc = int4larger,
+ mergefunc = int4larger
+);
+
-- invalid: nonstrict inverse with strict forward function
CREATE FUNCTION float8mi_n(float8, float8) RETURNS float8 AS
On Fri, Dec 5, 2014 at 8:46 PM, Stephen Frost <sfrost@snowman.net> wrote:
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..).
Initially I was thinking that first we should discuss if the design
and idea used in patch is sane, but now as you have asked and
even Robert has asked the same off list to me, I will take the
performance data next week (Another reason why I have not
taken any data is that still the work to push qualification down
to workers is left which I feel is quite important). However I still
think if I get some feedback on some of the basic things like below,
it would be good.
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.
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?
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).
- 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.
- 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 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.
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?
I think it is better to keep it dependent on other backend processes,
however for simplicity, I have kept it similar to max_wal_senders for now.
There's likely to be
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
performance tests.
One really interesting case would be parallel seq scans which are
executing against foreign tables/FDWs..
Sure.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Dec 5, 2014 at 8:43 PM, Stephen Frost <sfrost@snowman.net> wrote:
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-loadadditional_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.
This is exactly what I have done in patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Sat, Dec 6, 2014 at 12:27 AM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
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-loadadditional_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,
this would also eliminate all parallel overhead on small relations.
--
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Sat, Dec 6, 2014 at 10:43 AM, David Rowley <dgrowleyml@gmail.com> wrote:
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!
Thanks.
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.
I also think so, that it can be used in near future to enhance
and provide more value to the parallel scan feature. Thanks
for taking the initiative to do the leg-work for supporting
aggregates.
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.
Direction sounds to be right.
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.
Agreed.
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.
Don't you think that first we should stabilize the basic (target list
and quals that can be independently evaluated by workers) parallel
scan and then jump to do such enhancements?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
* 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
On Sat, Dec 6, 2014 at 5:37 PM, Stephen Frost <sfrost@snowman.net> wrote:
* 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?
Because for a simple sequence scan that much information is sufficient,
basically if we have scanrelid, target list, qual and then RTE (primarily
relOid), then worker can form and perform sequence scan.
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.
I think here you want to say that work allocation for workers should be
dynamic rather fixed which I think makes sense, however we can try
such an optimization after some initial performance data.
Perhaps for an 'explain analyze' we
return information about what workers actually *did* what, but that's a
whole different discussion.
Agreed.
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.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Dec 8, 2014 at 11:21 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
Sounds sensible. cpu_tuple_cost is already used for some other
purpose so not sure if it is right thing to override that parameter,
how about cpu_tuple_communication_cost or cpu_tuple_comm_cost.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Dec 8, 2014 at 11:27 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, 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.
If the statement contain quals that don't have volatile functions, then
they will be pushed down and the parallel can will be considered for
cost evaluation. I think eventually we might need some better way
to decide about which kind of functions are okay to be pushed.
I have also unified the way information is passed from master backend
to worker backends which is convert each node to string that has to be
passed and then later workers convert string to node, this has simplified
the related code.
I have taken performance data for different selectivity and complexity of
qual expressions, I understand that there will be other kind of scenario's
which we need to consider, however I think the current set of tests is good
place to start, please feel free to comment on kind of scenario's which you
want me to check
Performance Data
------------------------------
*m/c details*
IBM POWER-8 24 cores, 192 hardware threads
RAM = 492GB
*non-default settings in postgresql.conf*
max_connections=300
shared_buffers = 8GB
checkpoint_segments = 300
checkpoint_timeout = 30min
max_worker_processes=100
create table tbl_perf(c1 int, c2 char(1000));
30 million rows
------------------------
insert into tbl_perf values(generate_series(1,10000000),'aaaaa');
insert into tbl_perf values(generate_series(10000000,30000000),'aaaaa');
Function used in quals
-----------------------------------
A simple function which will perform some calculation and return
the value passed which can be used in qual condition.
create or replace function calc_factorial(a integer, fact_val integer)
returns integer
as $$
begin
perform (fact_val)!;
return a;
end;
$$ language plpgsql STABLE;
In below data,
num_workers - number of parallel workers configured using
parallel_seqscan_degree. 0, means it will execute sequence
scan and greater than 0 means parallel sequence scan.
exec_time - Execution Time given by Explain Analyze statement.
*Tests having quals containing function evaluation in qual*
*expressions.*
*Test-1*
*Query -* Explain analyze select c1 from tbl_perf where
c1 > calc_factorial(29700000,10) and c2 like '%aa%';
*Selection_criteria – *1% of rows will be selected
*num_workers* *exec_time (ms)* 0 229534 2 121741 4 67051 8 35607 16
24743
*Test-2*
*Query - *Explain analyze select c1 from tbl_perf where
c1 > calc_factorial(27000000,10) and c2 like '%aa%';
*Selection_criteria – *10% of rows will be selected
*num_workers* *exec_time (ms)* 0 226671 2 151587 4 93648 8 70540 16
55466
*Test-3*
*Query -* Explain analyze select c1 from tbl_perf
where c1 > calc_factorial(22500000,10) and c2 like '%aa%';
*Selection_criteria –* 25% of rows will be selected
*num_workers* *exec_time (ms)* 0 232673 2 197609 4 142686 8 111664 16
98097
*Tests having quals containing simple expressions in qual.*
*Test-4*
*Query - *Explain analyze select c1 from tbl_perf
where c1 > 29700000 and c2 like '%aa%';
*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
*Test-5*
*Query - *Explain analyze select c1 from tbl_perf
where c1 > 28500000 and c2 like '%aa%';
*Selection_criteria –* 5% of rows will be selected
*num_workers* *exec_time (ms)* 0 18906 2 13446 4 8970 8 7887 16 10403
*Test-6*
*Query -* Explain analyze select c1 from tbl_perf
where c1 > 27000000 and c2 like '%aa%';
*Selection_criteria – *10% of rows will be selected
*num_workers* *exec_time (ms)* 0 16132 2 23780 4 20275 8 11390 16
11418
Conclusion
------------------
1. Parallel workers help a lot when there is an expensive qualification
to evaluated, the more expensive the qualification the more better are
results.
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.
3. After certain point, increasing having more number of workers won't
help and rather have negative impact, refer Test-4.
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.
Thoughts and feedback for the current state of patch is welcome.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v2.patchapplication/octet-stream; name=parallel_seqscan_v2.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..91fbea5
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,359 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg);
+
+/*
+ * Indicate that an error came from a particular worker.
+ */
+static void
+worker_error_callback(void *arg)
+{
+ pid_t pid = * (pid_t *) arg;
+
+ errcontext("worker backend, pid %d", pid);
+}
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc)
+{
+ shm_mq_result res;
+ char msgtype;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int32 pid = 1234;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ --shmScan->num_shm_queues;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Initialize message buffer. */
+ initStringInfo(&msg);
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, false);
+ if (res != SHM_MQ_SUCCESS)
+ break;
+
+ /*
+ * Message-parsing routines operate on a null-terminated StringInfo,
+ * so we must construct one.
+ */
+ resetStringInfo(&msg);
+ enlargeStringInfo(&msg, nbytes);
+ msg.len = nbytes;
+ memcpy(msg.data, data, nbytes);
+ msg.data[nbytes] = '\0';
+ msgtype = pq_getmsgbyte(&msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback context;
+
+ /* Parse ErrorResponse or NoticeResponse. */
+ pq_parse_errornotice(&msg, &edata);
+
+ /*
+ * Limit the maximum error level to ERROR. We don't want
+ * a FATAL inside the backend worker to kill the user
+ * session.
+ */
+ if (edata.elevel > ERROR)
+ edata.elevel = ERROR;
+
+ /*
+ * Rethrow the error with an appropriate context method.
+ * On error, we need to ensure that master backend stop
+ * all other workers before propagating the error, so
+ * we need to pass the pid's of all workers, so that same
+ * can be done in error callback.
+ * XXX - For now, I am just sending some random number, this
+ * needs to be fixed.
+ */
+ context.callback = worker_error_callback;
+ context.arg = (void *) &pid;
+ context.previous = error_context_stack;
+ error_context_stack = &context;
+ ThrowErrorData(&edata);
+ error_context_stack = context.previous;
+
+ break;
+ }
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg.data[0], &msg.data[1], nbytes - 1);
+ break;
+ }
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(&msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description)
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(&msg); /* name */
+ (void) pq_getmsgint(&msg, 4); /* table OID */
+ (void) pq_getmsgint(&msg, 2); /* table attnum */
+ type_id = pq_getmsgint(&msg, 4); /* type OID */
+ (void) pq_getmsgint(&msg, 2); /* type length */
+ (void) pq_getmsgint(&msg, 4); /* typmod */
+ (void) pq_getmsgint(&msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(&msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ HeapTuple result;
+
+ result = form_result_tuple(resultState, tupdesc, &msg);
+ return result;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(&msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+
+ case 'Z':
+ {
+ /* Handle ReadyForQuery message. */
+ resultState->complete = true;
+ break;
+ }
+ default:
+ elog(WARNING, "unknown message type: %c (%zu bytes)",
+ msg.data[0], nbytes);
+ break;
+ }
+ }
+
+ /* Check whether the connection was broken prematurely. */
+ if (!resultState->complete)
+ ereport(ERROR,
+ (errcode(ERRCODE_CONNECTION_FAILURE),
+ errmsg("lost connection to worker process with PID %d",
+ pid)));
+
+ /*
+ * if we have exhausted data from all worker queues, then terminate
+ * processing data from queues.
+ */
+ if (shmScan->num_shm_queues <=0)
+ break;
+ else
+ {
+ ++queueId;
+ --shmScan->num_shm_queues;
+ resultState->has_row_description = false;
+ }
+ }
+
+ return NULL;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description)
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ return heap_form_tuple(tupdesc, values, isnull);
+}
\ No newline at end of file
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 332f04a..f158583 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -714,6 +714,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -910,6 +911,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1059,6 +1063,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1325,6 +1330,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Workers",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2142,6 +2157,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index e27c062..a28e74e 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 1319519..4f73a53 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -118,7 +118,7 @@ ExecScan(ScanState *node,
/*
* Fetch data from node
*/
- qual = node->ps.qual;
+ qual = node->ps.qualPushed ? NIL : node->ps.qual;
projInfo = node->ps.ps_ProjInfo;
econtext = node->ps.ps_ExprContext;
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..b04fae1
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,288 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * SeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(node->pss_currentShmScanDesc, node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not check
+ * are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * for parallel seq scan, qual is always pushed to be
+ * evaluated by backend worker.
+ */
+ parallelscanstate->ss.ps.qualPushed = true;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(parallelscanstate->ss.ss_currentRelation->rd_id,
+ node->scan.plan.targetlist,
+ node->scan.plan.qual,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->seg,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+
+ /*
+ * use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(node->num_workers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ /* detach from dynamic shared memory. */
+ dsm_detach(node->seg);
+}
+
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 53cfda5..131cfc5 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,22 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 449fdc3..dfd3b52 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 659daa2..0296323 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -106,6 +106,8 @@ int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +221,63 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * 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.
+ */
+ path->path.startup_cost = startup_cost / nWorkers;
+ path->path.total_cost = (startup_cost + run_cost) / nWorkers;
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..5245652
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,126 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 8f9ae4f..91a38e2 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -1133,6 +1147,71 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock)
+{
+ SeqScan *scan_plan;
+
+ /*
+ * Pass scan_relid as 1, this is okay for now as sequence scan worker
+ * is allowed to operate on just one relation.
+ * XXX - we should ideally get scanrelid from master backend.
+ */
+ scan_plan = make_seqscan(targetList,
+ scan_clauses,
+ 1);
+
+ scan_plan->startblock = startBlock;
+ scan_plan->endblock = endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3314,6 +3393,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index f752ecc..34cf588 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,59 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ AclMode required_access = ACL_SELECT;
+ RangeTblEntry *rte;
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+
+ rte = makeNode(RangeTblEntry);
+ rte->rtekind = RTE_RELATION;
+ rte->relid = workerstmt->relId;
+ rte->relkind = 'r';
+ rte->requiredPerms = required_access;
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) workerstmt->qual);
+
+ scan_plan = create_worker_seqscan_plan(workerstmt->targetList,
+ workerstmt->qual,
+ workerstmt->startBlock,
+ workerstmt->endBlock);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = list_make1(rte);
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, rte->relid);;
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 4d3fbca..bb8af32 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 319e8b2..ce3df40 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,37 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks (refer exec_worker_message).
+ */
+ pathnode->num_blocks_per_worker = rel->pages / nWorkers;
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..89d9aa2
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,607 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define SHM_PARALLEL_SCAN_QUEUE_SIZE 65536
+
+/*
+ * This structure is stored in the dynamic shared memory segment. We use
+ * it to determine whether all workers started up OK and successfully
+ * attached to their respective shared message queues.
+ */
+typedef struct
+{
+ slock_t mutex;
+ int workers_total;
+ int workers_attached;
+ int workers_ready;
+} shm_mq_header;
+
+/* Fixed-size data passed via our dynamic shared memory segment. */
+typedef struct worker_fixed_data
+{
+ Oid database_id;
+ Oid authenticated_user_id;
+ Oid current_user_id;
+ int sec_context;
+ NameData database;
+ NameData authenticated_user;
+} worker_fixed_data;
+
+/* Private state maintained by the launching backend for IPC. */
+typedef struct worker_info
+{
+ pid_t pid;
+ Oid current_user_id;
+ dsm_segment *seg;
+ BackgroundWorkerHandle *handle;
+ shm_mq_handle *responseq;
+ bool consumed;
+} worker_info;
+
+typedef struct
+{
+ int nworkers;
+ BackgroundWorkerHandle *handle[FLEXIBLE_ARRAY_MEMBER];
+} worker_state;
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PG_WORKER_MAGIC 0x50674267
+#define PG_WORKER_KEY_HDR_DATA 0
+#define PG_WORKER_KEY_FIXED_DATA 1
+#define PG_WORKER_KEY_RELID 2
+#define PG_WORKER_KEY_TARGETLIST 3
+#define PG_WORKER_KEY_QUAL 4
+#define PG_WORKER_KEY_BLOCKS 5
+#define PG_WORKER_FIXED_NKEYS 6
+
+void
+exec_worker_message(Datum) __attribute__((noreturn));
+
+static void
+setup_dynamic_shared_memory(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseq,
+ dsm_segment **segp, shm_mq_header **hdrp,
+ BlockNumber numBlocksPerWorker, int nWorkers);
+static worker_state *setup_backend_workers(dsm_segment *seg, int nworkers);
+static void cleanup_background_workers(dsm_segment *seg, Datum arg);
+static void
+wait_for_workers_to_become_ready(worker_state *wstate,
+ volatile shm_mq_header *hdr);
+static bool check_worker_status(worker_state *wstate);
+static void bkworker_sigterm_handler(SIGNAL_ARGS);
+
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseqp, dsm_segment **segp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ shm_mq_header *hdr;
+ worker_state *wstate;
+ int i;
+
+ /* Create dynamic shared memory and table of contents. */
+ setup_dynamic_shared_memory(relId, targetList, qual, responseqp,
+ segp, &hdr, numBlocksPerWorker, nWorkers);
+
+ /* Register backend workers. */
+ wstate = setup_backend_workers(*segp, nWorkers);
+
+ for (i = 0; i < nWorkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], wstate->handle[i]);
+
+ /* Wait for workers to become ready. */
+ wait_for_workers_to_become_ready(wstate, hdr);
+}
+
+/*
+ * Set up a dynamic shared memory segment.
+ *
+ * We set up a small control region that contains only a shm_mq_header,
+ * plus one region per message queue. There are as many message queues as
+ * the number of workers.
+ */
+static void
+setup_dynamic_shared_memory(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseqp,
+ dsm_segment **segp, shm_mq_header **hdrp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ Size segsize, targetlist_len, qual_len;
+ dsm_segment *seg;
+ shm_toc_estimator e;
+ shm_toc *toc;
+ worker_fixed_data *fdata;
+ Oid *reliddata;
+ char *targetlistdata;
+ char *targetlist_str;
+ char *qualdata;
+ char *qual_str;
+ int i;
+ shm_mq *mq;
+ shm_mq_header *hdr;
+ BlockNumber *num_blocks_per_worker;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /* Create dynamic shared memory and table of contents. */
+ shm_toc_initialize_estimator(&e);
+
+ shm_toc_estimate_chunk(&e, sizeof(shm_mq_header));
+
+ shm_toc_estimate_chunk(&e, sizeof(worker_fixed_data));
+
+ shm_toc_estimate_chunk(&e, sizeof(relId));
+
+ targetlist_str = nodeToString(targetList);
+ targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&e, targetlist_len);
+
+ qual_str = nodeToString(qual);
+ qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&e, qual_len);
+
+ shm_toc_estimate_chunk(&e, sizeof(BlockNumber));
+
+ for (i = 0; i < nWorkers; ++i)
+ shm_toc_estimate_chunk(&e, (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE);
+
+ shm_toc_estimate_keys(&e, PG_WORKER_FIXED_NKEYS + nWorkers);
+
+ segsize = shm_toc_estimate(&e);
+
+ seg = dsm_create(segsize);
+ toc = shm_toc_create(PG_WORKER_MAGIC, dsm_segment_address(seg),
+ segsize);
+
+ /* Set up the header region. */
+ hdr = shm_toc_allocate(toc, sizeof(shm_mq_header));
+ SpinLockInit(&hdr->mutex);
+ hdr->workers_total = nWorkers;
+ hdr->workers_attached = 0;
+ hdr->workers_ready = 0;
+ shm_toc_insert(toc, PG_WORKER_KEY_HDR_DATA, hdr);
+
+ /* Store fixed-size data in dynamic shared memory. */
+ fdata = shm_toc_allocate(toc, sizeof(worker_fixed_data));
+ fdata->database_id = MyDatabaseId;
+ fdata->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fdata->current_user_id, &fdata->sec_context);
+ namestrcpy(&fdata->database, get_database_name(MyDatabaseId));
+ namestrcpy(&fdata->authenticated_user,
+ GetUserNameFromId(fdata->authenticated_user_id));
+ shm_toc_insert(toc, PG_WORKER_KEY_FIXED_DATA, fdata);
+
+ /* Store scan relation id in dynamic shared memory. */
+ reliddata = shm_toc_allocate(toc, sizeof(Oid));
+ *reliddata = relId;
+ shm_toc_insert(toc, PG_WORKER_KEY_RELID, reliddata);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(toc, PG_WORKER_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(toc, PG_WORKER_KEY_QUAL, qualdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(toc, PG_WORKER_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Establish one message queue per worker in dynamic shared memory. */
+ for (i = 1; i <= nWorkers; ++i)
+ {
+ mq = shm_mq_create(shm_toc_allocate(toc, (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE),
+ (Size) SHM_PARALLEL_SCAN_QUEUE_SIZE);
+ shm_toc_insert(toc, PG_WORKER_FIXED_NKEYS + i, mq);
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i-1] = shm_mq_attach(mq, seg, NULL);
+ }
+
+ /* Return results to caller. */
+ *segp = seg;
+ *hdrp = hdr;
+}
+
+/*
+ * Register backend workers.
+ */
+static worker_state *
+setup_backend_workers(dsm_segment *seg, int nWorkers)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ worker_state *wstate;
+ int i;
+
+ /*
+ * We need the worker_state object and the background worker handles to
+ * which it points to be allocated in CurTransactionContext rather than
+ * ExprContext; otherwise, they'll be destroyed before the on_dsm_detach
+ * hooks run.
+ */
+ oldcontext = MemoryContextSwitchTo(CurTransactionContext);
+
+ /* Create worker state object. */
+ wstate = MemoryContextAlloc(TopTransactionContext,
+ offsetof(worker_state, handle) +
+ sizeof(BackgroundWorkerHandle *) * nWorkers);
+ wstate->nworkers = 0;
+
+ /*
+ * Arrange to kill all the workers if we abort before or after all workers
+ * are finished hooking themselves up to the dynamic shared memory segment.
+ *
+ * XXX - For killing workers, we need to have mechanism with which it can be
+ * done before aborting the transaction.
+ */
+
+ on_dsm_detach(seg, cleanup_background_workers,
+ PointerGetDatum(wstate));
+
+ /* Configure a worker. */
+ worker.bgw_flags =
+ BGWORKER_SHMEM_ACCESS | BGWORKER_BACKEND_DATABASE_CONNECTION;
+ worker.bgw_start_time = BgWorkerStart_ConsistentState;
+ worker.bgw_restart_time = BGW_NEVER_RESTART;
+ worker.bgw_main = exec_worker_message;
+ snprintf(worker.bgw_name, BGW_MAXLEN, "backend_worker");
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(seg));
+ /* set bgw_notify_pid, so we can detect if the worker stops */
+ worker.bgw_notify_pid = MyProcPid;
+
+ /* Register the workers. */
+ for (i = 0; i < nWorkers; ++i)
+ {
+ if (!RegisterDynamicBackgroundWorker(&worker, &wstate->handle[i]))
+ ereport(ERROR,
+ (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
+ errmsg("could not register background process"),
+ errhint("You may need to increase max_worker_processes.")));
+ ++wstate->nworkers;
+ }
+
+ /* All done. */
+ MemoryContextSwitchTo(oldcontext);
+ return wstate;
+}
+
+static void
+wait_for_workers_to_become_ready(worker_state *wstate,
+ volatile shm_mq_header *hdr)
+{
+ bool save_set_latch_on_sigusr1;
+ bool result = false;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ int workers_ready;
+
+ /* If all the workers are ready, we have succeeded. */
+ SpinLockAcquire(&hdr->mutex);
+ workers_ready = hdr->workers_ready;
+ SpinLockRelease(&hdr->mutex);
+ if (workers_ready >= wstate->nworkers)
+ {
+ result = true;
+ break;
+ }
+
+ /* If any workers (or the postmaster) have died, we have failed. */
+ if (!check_worker_status(wstate))
+ {
+ result = false;
+ break;
+ }
+
+ /* Wait to be signalled. */
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+
+ /* An interrupt may have occurred while we were waiting. */
+ CHECK_FOR_INTERRUPTS();
+
+ /* Reset the latch so we don't spin. */
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ if (!result)
+ ereport(ERROR,
+ (errcode(ERRCODE_INSUFFICIENT_RESOURCES),
+ errmsg("one or more background workers failed to start")));
+}
+
+static bool
+check_worker_status(worker_state *wstate)
+{
+ int n;
+
+ /* If any workers (or the postmaster) have died, we have failed. */
+ for (n = 0; n < wstate->nworkers; ++n)
+ {
+ BgwHandleStatus status;
+ pid_t pid;
+
+ status = GetBackgroundWorkerPid(wstate->handle[n], &pid);
+ /*if (status == BGWH_STOPPED || status == BGWH_POSTMASTER_DIED)*/
+ /*
+ * XXX - Do we need to consider BGWH_STOPPED status, if directly return
+ * false for BGWH_STOPPED, it could very well be possble that worker has
+ * exited after completing the work in which case the caller of this won't
+ * wait for other worker's status and main backend will lead to error
+ * whereas everything is normal for such a case.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ return false;
+ }
+
+ /* Otherwise, things still look OK. */
+ return true;
+}
+
+static void
+cleanup_background_workers(dsm_segment *seg, Datum arg)
+{
+ worker_state *wstate = (worker_state *) arg;
+
+ while (wstate->nworkers > 0)
+ {
+ --wstate->nworkers;
+ TerminateBackgroundWorker(wstate->handle[wstate->nworkers]);
+ }
+}
+
+
+/*
+ * exec_execute_message
+ *
+ * Process an "Execute" message for a portal
+ */
+void
+exec_worker_message(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ worker_fixed_data *fdata;
+ char *targetlistdata;
+ char *qualdata;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ int myworkernumber;
+ volatile shm_mq_header *hdr;
+ Oid *relId;
+ List *targetList = NIL;
+ List *qual = NIL;
+ PGPROC *registrant;
+ worker_stmt *workerstmt;
+ ResourceOwner saveBackgroundWorkerResourceOwner;
+ MemoryContext saveBackgroundWorkerContext;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, bkworker_sigterm_handler);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "backend_worker");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "backend worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /* Connect to the dynamic shared memory segment. */
+ seg = dsm_attach(DatumGetInt32(main_arg));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PG_WORKER_MAGIC, dsm_segment_address(seg));
+ if (toc == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("bad magic number in dynamic shared memory segment")));
+
+ /* Find data structures in dynamic shared memory. */
+ hdr = shm_toc_lookup(toc, PG_WORKER_KEY_HDR_DATA);
+ fdata = shm_toc_lookup(toc, PG_WORKER_KEY_FIXED_DATA);
+ relId = shm_toc_lookup(toc, PG_WORKER_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PG_WORKER_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PG_WORKER_KEY_QUAL);
+ num_blocks_per_worker = shm_toc_lookup(toc, PG_WORKER_KEY_BLOCKS);
+
+ /*
+ * Acquire a worker number.
+ *
+ * Our worker number gives our identity: there may be just one
+ * worker involved in this parallel operation, or there may be many.
+ */
+ SpinLockAcquire(&hdr->mutex);
+ myworkernumber = ++hdr->workers_attached;
+ SpinLockRelease(&hdr->mutex);
+ if (myworkernumber > hdr->workers_total)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many message queue testing workers already")));
+
+ mq = shm_toc_lookup(toc, PG_WORKER_FIXED_NKEYS + myworkernumber);
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = myworkernumber * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /*
+ * Indicate that we're fully initialized and ready to begin the main part
+ * of the parallel operation.
+ *
+ * Once we signal that we're ready, the user backend is entitled to assume
+ * that our on_dsm_detach callbacks will fire before we disconnect from
+ * the shared memory segment and exit. Generally, that means we must have
+ * attached to all relevant dynamic shared memory data structures by now.
+ */
+ SpinLockAcquire(&hdr->mutex);
+ ++hdr->workers_ready;
+ SpinLockRelease(&hdr->mutex);
+ registrant = BackendPidGetProc(MyBgworkerEntry->bgw_notify_pid);
+ if (registrant == NULL)
+ {
+ elog(DEBUG1, "registrant backend has exited prematurely");
+ proc_exit(1);
+ }
+ SetLatch(®istrant->procLatch);
+
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_shm_mq(mq, responseq);
+
+ /*
+ * Connection initialization will destroy the CurrentResourceOwner and
+ * CurrentMemoryContext as part of internal commit. This idea of
+ * internally starting whole new transactions is not good, but done
+ * elsewhere also, refer PortalRun.
+ */
+ saveBackgroundWorkerResourceOwner = CurrentResourceOwner;
+ saveBackgroundWorkerContext = CurrentMemoryContext;
+
+ /*
+ * Initialize our user and database ID based on the strings version of
+ * the data, and then go back and check that we actually got the database
+ * and user ID that we intended to get. We do this because it's not
+ * impossible for the process that started us to die before we get here,
+ * and the user or database could be renamed in the meantime. We don't
+ * want to latch on the wrong object by accident. There should probably
+ * be a variant of BackgroundWorkerInitializeConnection that accepts OIDs
+ * rather than strings.
+ */
+ BackgroundWorkerInitializeConnection(NameStr(fdata->database),
+ NameStr(fdata->authenticated_user));
+ if (fdata->database_id != MyDatabaseId ||
+ fdata->authenticated_user_id != GetAuthenticatedUserId())
+ ereport(ERROR,
+ (errmsg("user or database renamed during backend worker startup")));
+
+ CurrentResourceOwner = saveBackgroundWorkerResourceOwner;
+ CurrentMemoryContext = saveBackgroundWorkerContext;
+
+ /* Restore targetList and qual from main backend. */
+ targetList = (List *) stringToNode(targetlistdata);
+ qual = (List *) stringToNode(qualdata);
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fdata->current_user_id, fdata->sec_context);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->relId = *relId;
+ workerstmt->targetList = targetList;
+ workerstmt->qual = qual;
+ workerstmt->startBlock = start_block;
+
+ /* last worker should scan all the remaining blocks. */
+ if (myworkernumber == hdr->workers_total)
+ workerstmt->endBlock = InvalidBlockNumber;
+ else
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ ProcessCompletedNotifies();
+
+ /* Signal that we are done. */
+ ReadyForQuery(DestRemote);
+
+ proc_exit(1);
+}
+
+/*
+ * When we receive a SIGTERM, we set InterruptPending and ProcDiePending just
+ * like a normal backend. The next CHECK_FOR_INTERRUPTS() will do the right
+ * thing.
+ */
+static void
+bkworker_sigterm_handler(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
\ No newline at end of file
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..9d0c7c4 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -99,6 +99,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -831,6 +832,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..7de5e0e 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1132,6 +1133,105 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /* Make sure we are in a transaction command */
+ start_xact_command();
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal, NULL, 0, InvalidSnapshot);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+ (*receiver->rDestroy) (receiver);
+
+ PortalDrop(portal, false);
+
+ finish_xact_command();
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b1bff7f..6d855e3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -630,6 +630,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2445,6 +2447,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..50f7a27 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -497,6 +497,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index f2c7ca1..f88ef2e 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,7 +20,6 @@
#include "access/itup.h"
#include "access/tupdesc.h"
-
typedef struct HeapScanDescData
{
/* scan parameters */
@@ -105,4 +104,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..aa444bc
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,39 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ bool has_row_description;
+ bool complete;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41b13b2..7a615bc 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,11 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1021,6 +1023,9 @@ typedef struct PlanState
ProjectionInfo *ps_ProjInfo; /* info for doing tuple projection */
bool ps_TupFromTlist;/* state flag for processing set-valued
* functions in targetlist */
+ bool qualPushed; /* indicates that qual is pushed to backend
+ * worker, so no need to evaluate it after
+ * getting the tuple in main backend. */
} PlanState;
/* ----------------
@@ -1212,6 +1217,23 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ dsm_segment *seg;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index bc71fea..c48df6c 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 458eeb0..1ed9887 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -23,6 +23,7 @@
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +157,15 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Oid relId;
+ List *targetList;
+ List *qual;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 48203a0..e57c2d4 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 7116496..09fb141 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 75e2afb..a738c54 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -50,6 +50,7 @@ extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +69,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 26b17f5..901c792 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index afa5f9b..d2a2760 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 3fdc2cb..b382a27 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,9 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index 1e942c5..752bd16 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..19d6182
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,30 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Oid relId, List *targetList,
+ List *qual,
+ shm_mq_handle ***responseqp,
+ dsm_segment **segp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 60f7532..6087b5e 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index 47ff880..532d2db 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
* 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
On Fri, Dec 19, 2014 at 6:21 PM, Stephen Frost <sfrost@snowman.net> wrote:
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...
The main reason for higher cost of tuple communication is because at
this moment I have used an approach to pass the tuples which is
comparatively
less error prone and could be used as per existing FE/BE protocol.
To explain in brief, what is happening here is that currently worker backend
gets the tuple from page which it is deforms and send the same to master
backend via message queue, 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). The benefit of using this approach is that it works
as per current protocol message ('D') and as per our current executor code.
Now there could be couple of ways with which we can reduce the tuple
communication overhead.
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.
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 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.
The simple theory here is that there has to be certain limit
(in terms of number of parallel workers) till which there can
be benefit due to both of the above points and after which there
will be overhead (setting up so many workers even though they
are not required, then some additional wait by master backend
for non-helping workers to finish their work, then if there
are not enough CPU's available and may be others as well like
overusing I/O channel might also degrade the performance
rather than improving it).
In the above tests, it seems to me that the maximum benefit due to
'a' is realized upto 4~8 workers and the maximum benefit due to
'b' depends upon the complexity (time to evaluate) of expression.
That is the reason why we can see benefit's in Tests-1 ~ Test-3 above
8 parallel workers as well whereas for Tests-4 to Tests-6 it maximizes
at 8 workers and after that either there is no improvement or
degradation due to one or more reasons as explained in previous
paragraph.
I think important point which is mentioned by you as well is
that there should be a reasonably good cost model which can
account some or all of these things so that by using parallel
query user can achieve the benefit it provides and won't have
to pay the cost in which there is no or less benefit.
I am not sure that in first cut we can come up with a highly
robust cost model, but it should not be too weak that most
of the time user has to find the right tuning based on parameters
we are going to add. Based on my understanding and by referring
to existing literature, I will try to come up with the cost model
and then we can have a discussion if required whether that is good
enough for first cut or not.
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?
Not now, but sometime back I had read quite a few papers on parallelism,
I will refer some of them again before deciding the exact cost model
and might as well discuss about them.
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.
Got the point, I think we need to mention somewhere either in README or
in some file header.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.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? 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Dec 22, 2014 at 7:34 AM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
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?
That seems to be already there, before sending the tuple
to frontend, we already ensure to deform it (refer printtup()->
slot_getallattrs())
Other than that potential wrinkle, it seems like less work than a.
Here, I am assuming that you are mentioning about *pass the tuple*
directly approach; We also need to devise a new protocol message
and mechanism to directly pass the tuple via shared memory queues,
also I think currently we can send only the things via shared memory
queues which we can do via FE/BE protocol and we don't send tuples
directly to frontend. Apart from this, I am not sure how much benefit it
can give, because it will reduce one part of tuple communication, but still
the amount of data transferred will be almost same.
This is an area of improvement which needs more investigation and even
without this we can get benefit in many cases as shown upthread and
I think after that we can try to parallelize the aggregation (Simon Riggs
and
David Rowley have already worked out some infrastructure for the same)
that will surely give us good benefits. So I suggest it's better to focus
on
the remaining things with which this patch could be in a shape (in terms of
robustness/stability) where it can be accepted rather than trying to
optimize tuple communication which we can do later as well.
...
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 workersI'd think a good first estimate here would be to just use
effective_io_concurrency.
One thing we should be cautious about this parameter is that currently
it is mapped to number of pages that needs to prefetched, and using
it for deciding degree of parallelism could be slightly tricky, however I
will consider it while working on cost model.
Thanks for your suggestions.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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?
Thom
On 31 December 2014 at 14:20, Thom Brown <thom@linux.com> wrote:
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?
Another issue (FYI, pgbench2 initialised with: pgbench -i -s 100 -F 10
pgbench2):
➤ 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
The logs say:
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
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. Is that not possible to report?
--
Thom
On Wed, Dec 31, 2014 at 7:50 PM, Thom Brown <thom@linux.com> wrote:
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?
Yes, as the parallel workers are implemented based on dynamic
bgworkers, so it is dependent on max_worker_processes.
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?
I think one thing we could do minimize the chance of such an
error is set the value of parallel workers to be used for plan equal
to max_worker_processes if parallel_seqscan_degree is greater
than max_worker_processes. Even if we do this, still such an
error can come if user has registered bgworker before we could
start parallel plan execution.
Also, I noticed that where a table is partitioned, the plan isn't
parallelised:
Is this expected?
Yes, to keep the initial implementation simple, it allows the
parallel plan when there is single table in query.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Dec 31, 2014 at 9:46 PM, Thom Brown <thom@linux.com> wrote:
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 ms2014-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.
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 related
to BUFFERS is done by backend workers, master backend
doesn't read any pages, so it is not able to accumulate this
information.
Is that not possible to report?
It is not impossible to report such information, we can develop some
way to share such information between master backend and workers.
I think we can do this if required once the patch is more stablized.
Thanks for looking into patch and reporting the issues.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
I think one thing we could do minimize the chance of such an
error is set the value of parallel workers to be used for plan equal
to max_worker_processes if parallel_seqscan_degree is greater
than max_worker_processes. Even if we do this, still such an
error can come if user has registered bgworker before we could
start parallel plan execution.
Can we check the number of free bgworkers slots to set the max workers?
Regards,
Fabrízio Mello
--
Fabrízio de Royes Mello
Consultoria/Coaching PostgreSQL
Show quoted text
Timbira: http://www.timbira.com.br
Blog: http://fabriziomello.github.io
Linkedin: http://br.linkedin.com/in/fabriziomello
Twitter: http://twitter.com/fabriziomello
Github: http://github.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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 1 January 2015 at 17:59, 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 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.
+1
That sounds like exactly what's needed.
Thom
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
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
to BUFFERS is done by backend workers, master backend
doesn't read any pages, so it is not able to accumulate this
information.Is that not possible to report?
It is not impossible to report such information, we can develop some
way to share such information between master backend and workers.
I think we can do this if required once the patch is more stablized.
Ah great, as I think losing such information to this feature would be
unfortunate.
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:
# 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)
This is a quick plan, but if we tell it that it's allowed 8 background
workers:
# 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
It's significantly slower. I'd hope the planner would anticipate this and
decide, "I'm just gonna perform a single scan in this instance as it'll be
a lot quicker for this simple case." So at the moment
parallel_seqscan_degree seems to mean "You *must* use this many threads if
you can parallelise." Ideally we'd be saying "can use up to if necessary".
Thanks
Thom
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 memory
for longer period which causes it to hit OOM error, by the way
in your test does always master backend hits OOM or is it
random (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 parallel
communication (there is some discussion about the same
upthread), but it's still not there and that's why you
are seeing it to choose parallel plan when it shouldn't.
Currently in patch, if you set parallel_seqscan_degree, it
will most probably choose parallel plan only.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 2 January 2015 at 11:13, Amit Kapila <amit.kapila16@gmail.com> wrote:
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 memory
for longer period which causes it to hit OOM error, by the way
in your test does always master backend hits OOM or is it
random (either master or worker)
Just ran a few tests, and it appears to always be the master that hits OOM,
or at least I don't seem to be able to get an example of the worker hitting
it.
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 parallel
communication (there is some discussion about the same
upthread), but it's still not there and that's why you
are seeing it to choose parallel plan when it shouldn't.
Currently in patch, if you set parallel_seqscan_degree, it
will most probably choose parallel plan only.
Ah, okay. Great.
Thanks.
Thom
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
* 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 Mon, Jan 5, 2015 at 8:31 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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:
/messages/by-id/CA+TgmoYmp_=XcJEhvJZt9P8drBgW-pDpjHxBhZA79+M4o-CZQA@mail.gmail.com
Few things to note about this integrated patch are:
1. In this new patch, I have just integrated it with Robert's parallel_mode
patch and not done any further development or fixed known things
like changes in optimizer, prepare queries, etc. You might notice
that new patch has lesser size as compare to previous patch and the
reason is that there were some duplicate stuff between previous
version of parallel_seqscan patch and parallel_mode which I have
eliminated.
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.
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.
4. Sending ReadyForQuery
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Thu, Jan 8, 2015 at 5:12 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
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
by mistake.
4. Sending ReadyForQuery() after completely sending the tuples,
as that is required to know that all the tuples are received and I think
we should send the same on tuple queue rather than on error queue.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v3.patchapplication/octet-stream; name=parallel_seqscan_v3.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..91fbea5
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,359 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg);
+
+/*
+ * Indicate that an error came from a particular worker.
+ */
+static void
+worker_error_callback(void *arg)
+{
+ pid_t pid = * (pid_t *) arg;
+
+ errcontext("worker backend, pid %d", pid);
+}
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc)
+{
+ shm_mq_result res;
+ char msgtype;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int32 pid = 1234;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ --shmScan->num_shm_queues;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Initialize message buffer. */
+ initStringInfo(&msg);
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, false);
+ if (res != SHM_MQ_SUCCESS)
+ break;
+
+ /*
+ * Message-parsing routines operate on a null-terminated StringInfo,
+ * so we must construct one.
+ */
+ resetStringInfo(&msg);
+ enlargeStringInfo(&msg, nbytes);
+ msg.len = nbytes;
+ memcpy(msg.data, data, nbytes);
+ msg.data[nbytes] = '\0';
+ msgtype = pq_getmsgbyte(&msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback context;
+
+ /* Parse ErrorResponse or NoticeResponse. */
+ pq_parse_errornotice(&msg, &edata);
+
+ /*
+ * Limit the maximum error level to ERROR. We don't want
+ * a FATAL inside the backend worker to kill the user
+ * session.
+ */
+ if (edata.elevel > ERROR)
+ edata.elevel = ERROR;
+
+ /*
+ * Rethrow the error with an appropriate context method.
+ * On error, we need to ensure that master backend stop
+ * all other workers before propagating the error, so
+ * we need to pass the pid's of all workers, so that same
+ * can be done in error callback.
+ * XXX - For now, I am just sending some random number, this
+ * needs to be fixed.
+ */
+ context.callback = worker_error_callback;
+ context.arg = (void *) &pid;
+ context.previous = error_context_stack;
+ error_context_stack = &context;
+ ThrowErrorData(&edata);
+ error_context_stack = context.previous;
+
+ break;
+ }
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg.data[0], &msg.data[1], nbytes - 1);
+ break;
+ }
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(&msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description)
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(&msg); /* name */
+ (void) pq_getmsgint(&msg, 4); /* table OID */
+ (void) pq_getmsgint(&msg, 2); /* table attnum */
+ type_id = pq_getmsgint(&msg, 4); /* type OID */
+ (void) pq_getmsgint(&msg, 2); /* type length */
+ (void) pq_getmsgint(&msg, 4); /* typmod */
+ (void) pq_getmsgint(&msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(&msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ HeapTuple result;
+
+ result = form_result_tuple(resultState, tupdesc, &msg);
+ return result;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(&msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+
+ case 'Z':
+ {
+ /* Handle ReadyForQuery message. */
+ resultState->complete = true;
+ break;
+ }
+ default:
+ elog(WARNING, "unknown message type: %c (%zu bytes)",
+ msg.data[0], nbytes);
+ break;
+ }
+ }
+
+ /* Check whether the connection was broken prematurely. */
+ if (!resultState->complete)
+ ereport(ERROR,
+ (errcode(ERRCODE_CONNECTION_FAILURE),
+ errmsg("lost connection to worker process with PID %d",
+ pid)));
+
+ /*
+ * if we have exhausted data from all worker queues, then terminate
+ * processing data from queues.
+ */
+ if (shmScan->num_shm_queues <=0)
+ break;
+ else
+ {
+ ++queueId;
+ --shmScan->num_shm_queues;
+ resultState->has_row_description = false;
+ }
+ }
+
+ return NULL;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description)
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ return heap_form_tuple(tupdesc, values, isnull);
+}
\ No newline at end of file
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 71374cc..f46a1a3 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -39,43 +39,6 @@
*/
#define PARALLEL_ERROR_QUEUE_SIZE 16384
-/* Magic number for parallel context TOC. */
-#define PARALLEL_MAGIC 0x50477c7c
-
-/*
- * Magic numbers for parallel state sharing. Higher-level code should use
- * smaller values, leaving these very large ones for use by this module.
- */
-#define PARALLEL_KEY_FIXED UINT64CONST(0xFFFFFFFFFFFF0001)
-#define PARALLEL_KEY_ERROR_QUEUE UINT64CONST(0xFFFFFFFFFFFF0002)
-#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0003)
-#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0004)
-#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0005)
-#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
-#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0007)
-#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0008)
-
-/* Fixed-size parallel state. */
-typedef struct FixedParallelState
-{
- /* Fixed-size state that workers must restore. */
- Oid database_id;
- Oid authenticated_user_id;
- Oid current_user_id;
- int sec_context;
- PGPROC *parallel_master_pgproc;
- pid_t parallel_master_pid;
- BackendId parallel_master_backend_id;
-
- /* Entrypoint for parallel workers. */
- parallel_worker_main_type entrypoint;
-
- /* Track whether workers have attached. */
- slock_t mutex;
- int workers_expected;
- int workers_attached;
-} FixedParallelState;
-
/*
* Our parallel worker number. We initialize this to -1, meaning that we are
* not a parallel worker. In parallel workers, it will be set to a value >= 0
@@ -713,7 +676,7 @@ ParallelMain(Datum main_arg)
* Now that we have a resource owner, we can attach to the dynamic
* shared memory segment and read the table of contents.
*/
- seg = dsm_attach(DatumGetInt32(main_arg));
+ seg = dsm_attach(DatumGetUInt32(main_arg));
if (seg == NULL)
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
@@ -748,9 +711,12 @@ ParallelMain(Datum main_arg)
ParallelWorkerNumber * PARALLEL_ERROR_QUEUE_SIZE);
shm_mq_set_sender(mq, MyProc);
mqh = shm_mq_attach(mq, seg, NULL);
- pq_redirect_to_shm_mq(mq, mqh);
+ pq_save_shm_mq_info(mq, mqh);
+ pq_save_parallel_master_info(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+ /*pq_redirect_to_shm_mq(mq, mqh);
pq_set_parallel_master(fps->parallel_master_pid,
- fps->parallel_master_backend_id);
+ fps->parallel_master_backend_id);*/
/* Install an error-context callback. */
errctx.callback = ParallelErrorContext;
@@ -823,7 +789,7 @@ ParallelMain(Datum main_arg)
EndParallelWorkerTransaction();
/* Report success. */
- ReadyForQuery(DestRemote);
+ /*ReadyForQuery(DestRemote);*/
}
/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 8a0be5d..560b0d7 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -713,6 +713,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -909,6 +910,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1058,6 +1062,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1324,6 +1329,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Workers",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2141,6 +2156,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..f77a77f 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..67eda93 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -118,7 +118,7 @@ ExecScan(ScanState *node,
/*
* Fetch data from node
*/
- qual = node->ps.qual;
+ qual = node->ps.qualPushed ? NIL : node->ps.qual;
projInfo = node->ps.ps_ProjInfo;
econtext = node->ps.ps_ExprContext;
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..30570c9
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,291 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * SeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(node->pss_currentShmScanDesc, node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not check
+ * are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * for parallel seq scan, qual is always pushed to be
+ * evaluated by backend worker.
+ */
+ parallelscanstate->ss.ps.qualPushed = true;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(parallelscanstate->ss.ss_currentRelation->rd_id,
+ node->scan.plan.targetlist,
+ node->scan.plan.qual,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->pcxt,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+
+ /*
+ * use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(node->num_workers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+}
+
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 3cb81fc..5780df0 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,22 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index f12f2d5..6998e00 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -22,9 +22,13 @@
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
+static shm_mq *err_pq_mq = NULL;
+static shm_mq_handle *err_pq_mq_handle = NULL;
static bool pq_mq_busy = false;
static pid_t pq_mq_parallel_master_pid = 0;
static pid_t pq_mq_parallel_master_backend_id = InvalidBackendId;
+static pid_t save_pq_mq_parallel_master_pid = 0;
+static pid_t save_pq_mq_parallel_master_backend_id = InvalidBackendId;
static void mq_comm_reset(void);
static int mq_flush(void);
@@ -60,6 +64,30 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+void
+pq_save_shm_mq_info(shm_mq *mq, shm_mq_handle *mqh)
+{
+ err_pq_mq = mq;
+ err_pq_mq_handle = mqh;
+}
+
+void
+pq_redirect_to_err_shm_mq(void)
+{
+ Assert(err_pq_mq != NULL);
+ PqCommMethods = &PqCommMqMethods;
+ pq_mq = err_pq_mq;
+ pq_mq_handle = err_pq_mq_handle;
+ whereToSendOutput = DestRemote;
+ FrontendProtocol = PG_PROTOCOL_LATEST;
+}
+
+bool
+is_err_shm_mq_enabled(void)
+{
+ return err_pq_mq ? true : false;
+}
+
/*
* Arrange to SendProcSignal() to the parallel master each time we transmit
* message data via the shm_mq.
@@ -72,6 +100,21 @@ pq_set_parallel_master(pid_t pid, BackendId backend_id)
pq_mq_parallel_master_backend_id = backend_id;
}
+void
+pq_save_parallel_master_info(pid_t pid, BackendId backend_id)
+{
+ save_pq_mq_parallel_master_pid = pid;
+ save_pq_mq_parallel_master_backend_id = backend_id;
+}
+
+void
+pq_set_parallel_master_from_info(void)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = save_pq_mq_parallel_master_pid;
+ pq_mq_parallel_master_backend_id = save_pq_mq_parallel_master_backend_id;
+}
+
static void
mq_comm_reset(void)
{
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 020558b..dedce1f 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -106,6 +106,8 @@ int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +221,63 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * 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.
+ */
+ path->path.startup_cost = startup_cost / nWorkers;
+ path->path.total_cost = (startup_cost + run_cost) / nWorkers;
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..5245652
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,126 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 655be81..1c7f640 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -1133,6 +1147,71 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock)
+{
+ SeqScan *scan_plan;
+
+ /*
+ * Pass scan_relid as 1, this is okay for now as sequence scan worker
+ * is allowed to operate on just one relation.
+ * XXX - we should ideally get scanrelid from master backend.
+ */
+ scan_plan = make_seqscan(targetList,
+ scan_clauses,
+ 1);
+
+ scan_plan->startblock = startBlock;
+ scan_plan->endblock = endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3314,6 +3393,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9cbbcfb..6c8c3f0 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,59 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ AclMode required_access = ACL_SELECT;
+ RangeTblEntry *rte;
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+
+ rte = makeNode(RangeTblEntry);
+ rte->rtekind = RTE_RELATION;
+ rte->relid = workerstmt->relId;
+ rte->relkind = 'r';
+ rte->requiredPerms = required_access;
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) workerstmt->qual);
+
+ scan_plan = create_worker_seqscan_plan(workerstmt->targetList,
+ workerstmt->qual,
+ workerstmt->startBlock,
+ workerstmt->endBlock);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = list_make1(rte);
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, rte->relid);;
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 7703946..3a44aef 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..2ca1707 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,37 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks (refer exec_worker_message).
+ */
+ pathnode->num_blocks_per_worker = rel->pages / nWorkers;
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..028f34e
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,226 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PG_WORKER_KEY_RELID 0
+#define PG_WORKER_KEY_TARGETLIST 1
+#define PG_WORKER_KEY_QUAL 2
+#define PG_WORKER_KEY_BLOCKS 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+
+void exec_worker_message(dsm_segment *seg, shm_toc *toc);
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ int i;
+ Size targetlist_len, qual_len;
+ BlockNumber *num_blocks_per_worker;
+ Oid *reliddata;
+ char *targetlistdata;
+ char *targetlist_str;
+ char *qualdata;
+ char *qual_str;
+ char *tuple_queue_space;
+ ParallelContext *pcxt;
+ shm_mq *mq;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(exec_worker_message, nWorkers);
+
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(relId));
+
+ targetlist_str = nodeToString(targetList);
+ targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, targetlist_len);
+
+ qual_str = nodeToString(qual);
+ qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, qual_len);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(BlockNumber));
+
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * nWorkers);
+
+ /* 5 keys for parallel seq. scan specific data. */
+ shm_toc_estimate_keys(&pcxt->estimator, 5);
+
+ InitializeParallelDSM(pcxt);
+
+ /* Store scan relation id in dynamic shared memory. */
+ reliddata = shm_toc_allocate(pcxt->toc, sizeof(Oid));
+ *reliddata = relId;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_RELID, reliddata);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(pcxt->toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(pcxt->toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_QUAL, qualdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(pcxt->toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(pcxt);
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], pcxt->worker[i].bgwhandle);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+
+/*
+ * exec_worker_message
+ *
+ * Execute the work assigned to a worker by master backend.
+ */
+void
+exec_worker_message(dsm_segment *seg, shm_toc *toc)
+{
+ char *targetlistdata;
+ char *qualdata;
+ char *tuple_queue_space;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ FixedParallelState *fps;
+ Oid *relId;
+ List *targetList = NIL;
+ List *qual = NIL;
+ worker_stmt *workerstmt;
+
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ relId = shm_toc_lookup(toc, PG_WORKER_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PG_WORKER_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PG_WORKER_KEY_QUAL);
+ num_blocks_per_worker = shm_toc_lookup(toc, PG_WORKER_KEY_BLOCKS);
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = (ParallelWorkerNumber + 1) * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_shm_mq(mq, responseq);
+
+ /* Restore targetList and qual from main backend. */
+ targetList = (List *) stringToNode(targetlistdata);
+ qual = (List *) stringToNode(qualdata);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->relId = *relId;
+ workerstmt->targetList = targetList;
+ workerstmt->qual = qual;
+ workerstmt->startBlock = start_block;
+
+ /*
+ * last worker should scan all the remaining blocks.
+ *
+ * XXX - It is possible that expected number of workers
+ * won't get started, so to handle such cases master
+ * backend should scan remaining blocks.
+ */
+ if ((ParallelWorkerNumber + 1) == fps->workers_expected)
+ workerstmt->endBlock = InvalidBlockNumber;
+ else
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 65d5fac..6c7d89a 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7b1e8f6..d345d4c 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -56,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1133,6 +1134,100 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal, NULL, 0, InvalidSnapshot);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+ (*receiver->rDestroy) (receiver);
+
+ PortalDrop(portal, false);
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/error/elog.c b/src/backend/utils/error/elog.c
index 13395e3..a373f6b 100644
--- a/src/backend/utils/error/elog.c
+++ b/src/backend/utils/error/elog.c
@@ -67,6 +67,7 @@
#include "access/xact.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
#include "mb/pg_wchar.h"
#include "miscadmin.h"
#include "postmaster/postmaster.h"
@@ -236,6 +237,13 @@ errstart(int elevel, const char *filename, int lineno,
bool output_to_client = false;
int i;
+ /* redirect errors to error shared memory queue. */
+ if (is_err_shm_mq_enabled() && elevel >= ERROR)
+ {
+ pq_redirect_to_err_shm_mq();
+ pq_set_parallel_master_from_info();
+ }
+
/*
* Check some cases in which we want to promote an error into a more
* severe error. None of this logic applies for non-error messages.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d9bfa25..9319f65 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -630,6 +630,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2445,6 +2447,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..50f7a27 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -497,6 +497,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index b651218..c50dd7b 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -41,8 +41,47 @@ typedef struct ParallelContext
ParallelWorkerInfo *worker;
} ParallelContext;
+/* Magic number for parallel context TOC. */
+#define PARALLEL_MAGIC 0x50477c7c
+
+/*
+ * Magic numbers for parallel state sharing. Higher-level code should use
+ * smaller values, leaving these very large ones for use by this module.
+ */
+#define PARALLEL_KEY_FIXED UINT64CONST(0xFFFFFFFFFFFF0001)
+#define PARALLEL_KEY_ERROR_QUEUE UINT64CONST(0xFFFFFFFFFFFF0002)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0008)
+
+/* Fixed-size parallel state. */
+typedef struct FixedParallelState
+{
+ /* Fixed-size state that workers must restore. */
+ Oid database_id;
+ Oid authenticated_user_id;
+ Oid current_user_id;
+ int sec_context;
+ PGPROC *parallel_master_pgproc;
+ pid_t parallel_master_pid;
+ BackendId parallel_master_backend_id;
+
+ /* Entrypoint for parallel workers. */
+ parallel_worker_main_type entrypoint;
+
+ /* Track whether workers have attached. */
+ slock_t mutex;
+ int workers_expected;
+ int workers_attached;
+} FixedParallelState;
+
extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExtension(char *library_name,
char *function_name, int nworkers);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..bde6df0 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,7 +20,6 @@
#include "access/itup.h"
#include "access/tupdesc.h"
-
typedef struct HeapScanDescData
{
/* scan parameters */
@@ -105,4 +104,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..aa444bc
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,39 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ bool has_row_description;
+ bool complete;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(ShmScanDesc shmScan, worker_result resultState,
+ shm_mq_handle **responseq, TupleDesc tupdesc);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index ad7589d..2186d60 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,7 +17,13 @@
#include "storage/shm_mq.h"
extern void pq_redirect_to_shm_mq(shm_mq *, shm_mq_handle *);
+extern void pq_save_shm_mq_info(shm_mq *mq, shm_mq_handle *mqh);
+extern void pq_redirect_to_err_shm_mq(void);
+extern bool is_err_shm_mq_enabled(void);
+
extern void pq_set_parallel_master(pid_t pid, BackendId backend_id);
+extern void pq_save_parallel_master_info(pid_t pid, BackendId backend_id);
+extern void pq_set_parallel_master_from_info(void);
extern void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41288ed..a7263bd 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,12 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1021,6 +1024,9 @@ typedef struct PlanState
ProjectionInfo *ps_ProjInfo; /* info for doing tuple projection */
bool ps_TupFromTlist;/* state flag for processing set-valued
* functions in targetlist */
+ bool qualPushed; /* indicates that qual is pushed to backend
+ * worker, so no need to evaluate it after
+ * getting the tuple in main backend. */
} PlanState;
/* ----------------
@@ -1212,6 +1218,23 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..b6f1493 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index b1dfa85..5777271 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -23,6 +23,7 @@
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +157,15 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Oid relId;
+ List *targetList;
+ List *qual;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 316c9ce..3354398 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..576add5 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..b1161bd 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -50,6 +50,7 @@ extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +69,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..32c3e0d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 082f7d7..ef5a320 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,9 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..91ddffe 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..8813b6d
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,30 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Oid relId, List *targetList,
+ List *qual,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 0a350fd..02cf518 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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.
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
* 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 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.
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 Fri, Jan 9, 2015 at 1:02 AM, 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 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.
This is something how other commercial database handles such a
situation.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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:
Costing:
In one of the paper's [1]http://i.stanford.edu/pub/cstr/reports/cs/tr/96/1570/CS-TR-96-1570.pdf 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.
Execution:
Most other databases does partition level scan for partition on
different disks by each individual parallel worker. However,
it seems amazon dynamodb [2]http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/QueryAndScan.html#QueryAndScanParallelScan 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.
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.
Let me know your opinion about the same?
I am planning to proceed with above ideas to strengthen the patch
in absence of any objection or better ideas.
[1]: http://i.stanford.edu/pub/cstr/reports/cs/tr/96/1570/CS-TR-96-1570.pdf
[2]: http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/QueryAndScan.html#QueryAndScanParallelScan
http://docs.aws.amazon.com/amazondynamodb/latest/developerguide/QueryAndScan.html#QueryAndScanParallelScan
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
* 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 showing it'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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 possible join permutation, or attempt to parallelize?
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Jan 9, 2015 at 10:54 PM, Stephen Frost <sfrost@snowman.net> wrote:
* 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.
Actually the main factor to decide whether a parallel plan will be
selected or not will be based on selectivity and cpu_tuple_comm_cost,
parallel_startup_cost is mainly to prevent the cases where user
has set parallel_seqscan_degree, but the table is small enough
(letus say 10,000 tuples) that it doesn't need parallelism. If you are
worried by default cost parameter's, then I think those still needs
to be decided based on certain experiments.
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.
The main idea behind providing a parameter like parallel_seqscan_degree
is such that it will try to use that many number of workers for a single
parallel operation (intra-node parallelism) and incase we have to perform
inter-node parallelism than having such an parameter means that each
node can use that many number of parallel worker. For example we have
to parallelize scan as well as sort (Select * from t1 order by c1), and
parallel_degree is specified as 2, then each of the scan and sort can use
2 parallel workers each.
This is somewhat similar to the concept how degree of parallelism (DOP)
works in other databases. Refer case of Oracle [1]http://docs.oracle.com/cd/A57673_01/DOC/server/doc/A48506/pqoconce.htm (Setting Degree of
Parallelism).
I don't deny the fact that it will be a idea worth exploring to make
optimizer
more smart for deciding parallel plans, but it seems to me it is an advanced
topic which will be more valuable when we will try to parallelize joins or
other
similar stuff and even most papers talk about it in those regards only.
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.
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.
I am not clear, do you expect anything different in execution strategy
than what I have mentioned or does that sound reasonable to you?
[1]: http://docs.oracle.com/cd/A57673_01/DOC/server/doc/A48506/pqoconce.htm
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Sat, Jan 10, 2015 at 2:45 AM, Stefan Kaltenbrunner
<stefan@kaltenbrunner.cc> wrote:
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
query is going to take many hours if run sequentially and it could
be finished in minutes if run with 16 parallel workers, now let us
say during execution if there are less than 30% of parallel workers
available it might not be acceptable to user and he would like to
rather wait for some time and again run the query and if he wants
to run query even if 2 workers are available, he can choose not
to such a parameter.
Having said that, I also feel this doesn't seem to be an important case
to introduce a new parameter and such a behaviour. I have mentioned,
because it came across my eyes how some other databases handle
such a situation. Lets forget this suggestion if we can't imagine any
use of such a parameter.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
* 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:
/messages/by-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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sun, Jan 11, 2015 at 9:09 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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?
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
by backend worker. In current form, the patch needs to know
if myworker is the last worker (and I have used workers_expected
to achieve the same, I know that is not the right thing but I need
something similar if we decide to do in the way I have proposed),
so that it can scan all the remaining blocks.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
* 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Jan 12, 2015 at 3:30 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
Consider the case of inter-node parallelism, in such cases master
backend will have 4 responsibilities (scan relation, receive tuples
from other workers, send tuples to other workers, send tuples to
frontend) if we make it act like a worker.
For example
Select * from t1 Order By c1;
Now here first it needs to perform parallel sequential scan and then
fed the tuples from scan to another parallel worker which is doing sort.
It seems to me that master backend could starve few resources doing
all the work in an optimized way. As an example, one case could be
master backend read one page in memory (shared buffers) and then
read one tuple and apply the qualification and in the mean time the
queues on which it needs to receive got filled and it becomes busy
fetching tuples from those queues, now the page which it has read from
disk will be pinned in shared buffers for a longer time and even if we
release such a page, it has to be read again. OTOH, if master backend
would choose to read all the tuples from a page before checking the status
of queues, it can lead to lot of data piled up in queues.
I think there can be more such scenarios where getting many things
done by master backend can turn out to have negative impact.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Jan 12, 2015 at 3:27 AM, Robert Haas <robertmhaas@gmail.com> 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.
So what should happen if no workers are available?
I don't think we can change the plan to a non-parallel at that
stage.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 to get 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 sure we're not painting ourselves in a corner.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 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 7:25 AM, 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.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 forgot to mention that a running task queue can provide metrics such as
current utilization, current average throughput, current queue length and
estimated queue wait time. These can become dynamic cost factors in
deciding whether to parallelize.
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 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).
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 seems
to me that for initial version even if the workers are not shared,
then also it is meaningful to make parallel sequential scan work.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Jan 14, 2015 at 9:12 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
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'tparallelize
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 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.
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 workerto 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 andreported on and becomes considerably more predictable.
Yeah, it is good idea to maintain shared worker pool, but it seems
to me that for initial version even if the workers are not shared,
then also it is meaningful to make parallel sequential scan work.With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
--
Best Wishes,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 to cause 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, during an index scan you'd really like to get IO requests for heap blocks started in the background while the backend is focused on the mechanics of the index scan itself. While this could be done with the stuff Robert has written I have to wonder if it'd be a lot more efficient to use fadvise or AIO. Or perhaps it would just be better to deal with an entire index page (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 start thinking about how to optimize it.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 14, 2015 at 9:30 AM, Ashutosh Bapat <
ashutosh.bapat@enterprisedb.com> wrote:
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
with what Robert has suggested, but instead of block-by-block,
you seem to be suggesting of doing it in chunks (where chunk size
is not clear), however the only point against this is that such a
strategy for parallel sequence scan could lead to random scans
which can hurt the operation badly. Nonetheless, I will think more
on this lines of making work distribution dynamic so that we can
ensure that all workers can be kept busy.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Jan 12, 2015 at 3:25 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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
switching, but this way we need to do it separately for 'T', 'D', and
'C' messages.
I have moved this patch to next CF as apart from above still I
have to work on execution strategy and optimizer related changes
as discussed in this thread
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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?
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Jan 17, 2015 at 10:09 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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:
/messages/by-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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v4.patchapplication/octet-stream; name=parallel_seqscan_v4.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..1afac59 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -243,7 +243,19 @@ SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist, int16 *formats)
pq_sendint(&buf, 0, 2);
}
}
- pq_endmessage(&buf);
+
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
}
/*
@@ -371,7 +383,18 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
}
}
- pq_endmessage(&buf);
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..8c7dedb
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,373 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+ workerResult->num_shm_queues = nWorkers;
+ workerResult->has_row_description = palloc0(sizeof(bool) * nWorkers);
+ workerResult->queue_detached = palloc0(sizeof(bool) * nWorkers);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap)
+{
+ shm_mq_result res;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ if (!resultState->all_queues_detached)
+ {
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+
+ /*
+ * Don't fetch from detached queue. This loop could continue
+ * forever, if we reach a situation such that all queue's are
+ * detached, however we won't reach here if that is the case.
+ */
+ while (resultState->queue_detached[queueId])
+ {
+ ++queueId;
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+ }
+
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, true);
+ if (res == SHM_MQ_DETACHED)
+ {
+ /*
+ * mark the queue that got detached, so that we don't
+ * try to fetch from it again.
+ */
+ resultState->queue_detached[queueId] = true;
+ resultState->has_row_description[queueId] = false;
+ --resultState->num_shm_queues;
+ /*
+ * if we have exhausted data from all worker queues, then don't
+ * process data from queues.
+ */
+ if (resultState->num_shm_queues <= 0)
+ resultState->all_queues_detached = true;
+ break;
+ }
+ else if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ bool rettuple;
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ rettuple = HandleParallelTupleMessage(resultState, tupdesc, &msg, queueId);
+ pfree(msg.data);
+ if (rettuple)
+ {
+ *fromheap = false;
+ return resultState->tuple;
+ }
+ }
+ }
+ }
+
+ /*
+ * if we have checked all the message queue's and didn't find
+ * any message or we have already fetched all the data from queue's,
+ * then it's time to fetch directly from heap. Reset the current
+ * queue as the first queue from which we need to receive tuples.
+ */
+ if ((queueId == shmScan->num_shm_queues - 1 ||
+ resultState->all_queues_detached) &&
+ !resultState->all_heap_fetched)
+ {
+ HeapTuple tuple;
+ shmScan->ss_cqueue = 0;
+ tuple = heap_getnext(scanDesc, direction);
+ if (tuple)
+ {
+ *fromheap = true;
+ return tuple;
+ }
+ else if (tuple == NULL && resultState->all_queues_detached)
+ break;
+ else
+ resultState->all_heap_fetched = true;
+ }
+ else if (resultState->all_queues_detached &&
+ resultState->all_heap_fetched)
+ break;
+
+ /* check the data in next queue. */
+ ++queueId;
+ }
+
+ return NULL;
+}
+
+/*
+ * HandleParallelTupleMessage -
+ * Handle a single tuple related protocol message received from
+ * a single parallel worker.
+ */
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ char msgtype;
+ bool rettuple = false;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description[queueId])
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description[queueId] = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(msg); /* name */
+ (void) pq_getmsgint(msg, 4); /* table OID */
+ (void) pq_getmsgint(msg, 2); /* table attnum */
+ type_id = pq_getmsgint(msg, 4); /* type OID */
+ (void) pq_getmsgint(msg, 2); /* type length */
+ (void) pq_getmsgint(msg, 4); /* typmod */
+ (void) pq_getmsgint(msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ resultState->tuple = form_result_tuple(resultState, tupdesc, msg, queueId);
+ rettuple = true;
+ break;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+ default:
+ elog(WARNING, "unknown message type: %c", msg->data[0]);
+ break;
+ }
+
+ return rettuple;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ HeapTuple tuple;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description[queueId])
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ tuple = heap_form_tuple(tupdesc, values, isnull);
+
+ /*
+ * Release locally palloc'd space. XXX would probably be good to pfree
+ * values of pass-by-reference datums, as well.
+ */
+ pfree(values);
+ pfree(isnull);
+
+ return tuple;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 8a0be5d..bb581a8 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -713,6 +713,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -909,6 +910,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1058,6 +1062,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1324,6 +1329,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Worker",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2141,6 +2156,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..f77a77f 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..39c624d 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,8 +191,17 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification as for them the same is done by backend
+ * worker. This case will happen only for parallel query where we push
+ * down the qualification.
+ * XXX - We can do this optimization for projection as well, but for
+ * now it is okay, as we don't allow parallel query if there are
+ * expressions involved in target list.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..5200df5
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,318 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ParallelSeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc, node->pss_currentShmScanDesc,
+ node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ direction, &fromheap);
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * shm_beginscan/heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+ BlockNumber end_block;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /*
+ * If we are just doing EXPLAIN (ie, aren't going to run the plan), stop
+ * here, no need to start workers.
+ */
+ if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
+ return parallelscanstate;
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(parallelscanstate->ss.ss_currentRelation->rd_id,
+ node->scan.plan.targetlist,
+ node->scan.plan.qual,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->pcxt,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+ /* Initialize the blocks to be scanned by master backend. */
+ end_block = (parallelscanstate->pcxt->nworkers + 1) *
+ node->num_blocks_per_worker;
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock =
+ end_block - node->num_blocks_per_worker;
+ /*
+ * As master backend is the last backend to scan the blocks, it
+ * should scan all the blocks.
+ */
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->endblock = InvalidBlockNumber;
+
+ /* Set the scan limits for master backend. */
+ heap_setscanlimits(parallelscanstate->ss.ss_currentScanDesc,
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock,
+ (parallelscanstate->ss.ss_currentScanDesc->rs_nblocks -
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock));
+
+ /*
+ * Use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * Number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(parallelscanstate->pcxt->nworkers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ parallelscanstate->pcxt->nworkers);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt)
+ {
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 3cb81fc..5780df0 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,22 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index f12f2d5..cfab8b5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -26,6 +26,8 @@ static bool pq_mq_busy = false;
static pid_t pq_mq_parallel_master_pid = 0;
static pid_t pq_mq_parallel_master_backend_id = InvalidBackendId;
+static shm_mq_handle *pq_mq_tuple_handle = NULL;
+
static void mq_comm_reset(void);
static int mq_flush(void);
static int mq_flush_if_writable(void);
@@ -61,6 +63,26 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
}
/*
+ * Arrange to send some frontend/backend protocol messages to a shared-memory
+ * tuple message queue.
+ */
+void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh)
+{
+ pq_mq_tuple_handle = mqh;
+}
+
+/*
+ * Check if tuples can be sent through tuple shared-memory
+ * message queue.
+ */
+bool
+is_tuple_shm_mq_enabled(void)
+{
+ return pq_mq_tuple_handle ? true : false;
+}
+
+/*
* Arrange to SendProcSignal() to the parallel master each time we transmit
* message data via the shm_mq.
*/
@@ -161,6 +183,42 @@ mq_putmessage(char msgtype, const char *s, size_t len)
return 0;
}
+/*
+ * Transmit a libpq protocol message to the shared memory message queue
+ * via pq_mq_tuple_handle. We don't include a length word, because the
+ * receiver will know the length of the message from shm_mq_receive().
+ */
+int
+mq_putmessage_direct(char msgtype, const char *s, size_t len)
+{
+ shm_mq_iovec iov[2];
+ shm_mq_result result;
+
+ iov[0].data = &msgtype;
+ iov[0].len = 1;
+ iov[1].data = s;
+ iov[1].len = len;
+
+ Assert(pq_mq_tuple_handle != NULL);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_tuple_handle, iov, 2, true);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ Assert(result == SHM_MQ_SUCCESS || result == SHM_MQ_DETACHED);
+ if (result != SHM_MQ_SUCCESS)
+ return EOF;
+ return 0;
+}
+
static void
mq_putmessage_noblock(char msgtype, const char *s, size_t len)
{
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 020558b..4abfd25 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +227,73 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..5245652
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,126 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 655be81..1c7f640 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -1133,6 +1147,71 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock)
+{
+ SeqScan *scan_plan;
+
+ /*
+ * Pass scan_relid as 1, this is okay for now as sequence scan worker
+ * is allowed to operate on just one relation.
+ * XXX - we should ideally get scanrelid from master backend.
+ */
+ scan_plan = make_seqscan(targetList,
+ scan_clauses,
+ 1);
+
+ scan_plan->startblock = startBlock;
+ scan_plan->endblock = endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3314,6 +3393,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9cbbcfb..d2b1621 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,71 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ AclMode required_access = ACL_SELECT;
+ RangeTblEntry *rte;
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ rte = makeNode(RangeTblEntry);
+ rte->rtekind = RTE_RELATION;
+ rte->relid = workerstmt->relId;
+ rte->relkind = 'r';
+ rte->requiredPerms = required_access;
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) workerstmt->qual);
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, workerstmt->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ scan_plan = create_worker_seqscan_plan(workerstmt->targetList,
+ workerstmt->qual,
+ workerstmt->startBlock,
+ workerstmt->endBlock);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = list_make1(rte);
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, rte->relid);;
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 7703946..3a44aef 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..538e612 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,41 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks. We always consider master backend
+ * as last worker because it will first try to get the tuples
+ * scanned by other workers. For calculation of number of blocks
+ * per worker, an additional worker needs to be consider for
+ * master backend.
+ */
+ pathnode->num_blocks_per_worker = rel->pages / (nWorkers + 1);
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..d52d1b6
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,224 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PG_WORKER_KEY_RELID 0
+#define PG_WORKER_KEY_TARGETLIST 1
+#define PG_WORKER_KEY_QUAL 2
+#define PG_WORKER_KEY_BLOCKS 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+
+static void exec_worker_message(dsm_segment *seg, shm_toc *toc);
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ int i;
+ Size targetlist_len, qual_len;
+ BlockNumber *num_blocks_per_worker;
+ Oid *reliddata;
+ char *targetlistdata;
+ char *targetlist_str;
+ char *qualdata;
+ char *qual_str;
+ char *tuple_queue_space;
+ ParallelContext *pcxt;
+ shm_mq *mq;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(exec_worker_message, nWorkers);
+
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(relId));
+
+ targetlist_str = nodeToString(targetList);
+ targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, targetlist_len);
+
+ qual_str = nodeToString(qual);
+ qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, qual_len);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(BlockNumber));
+
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * nWorkers);
+
+ /* 5 keys for parallel seq. scan specific data. */
+ shm_toc_estimate_keys(&pcxt->estimator, 5);
+
+ InitializeParallelDSM(pcxt);
+
+ /* Store scan relation id in dynamic shared memory. */
+ reliddata = shm_toc_allocate(pcxt->toc, sizeof(Oid));
+ *reliddata = relId;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_RELID, reliddata);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(pcxt->toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(pcxt->toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_QUAL, qualdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(pcxt->toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(pcxt);
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], pcxt->worker[i].bgwhandle);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+
+/*
+ * exec_worker_message
+ *
+ * Execute the work assigned to a worker by master backend.
+ */
+void
+exec_worker_message(dsm_segment *seg, shm_toc *toc)
+{
+ char *targetlistdata;
+ char *qualdata;
+ char *tuple_queue_space;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ Oid *relId;
+ List *targetList = NIL;
+ List *qual = NIL;
+ worker_stmt *workerstmt;
+
+ relId = shm_toc_lookup(toc, PG_WORKER_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PG_WORKER_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PG_WORKER_KEY_QUAL);
+ num_blocks_per_worker = shm_toc_lookup(toc, PG_WORKER_KEY_BLOCKS);
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = (ParallelWorkerNumber + 1) * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_tuple_shm_mq(responseq);
+
+ /* Restore targetList and qual passed by main backend. */
+ targetList = (List *) stringToNode(targetlistdata);
+ qual = (List *) stringToNode(qualdata);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->relId = *relId;
+ workerstmt->targetList = targetList;
+ workerstmt->qual = qual;
+ workerstmt->startBlock = start_block;
+
+ /*
+ * Last worker should scan all the remaining blocks.
+ *
+ * XXX - It is possible that expected number of workers
+ * won't get started, so to handle such cases master
+ * backend should scan remaining blocks.
+ */
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 47ed84c..994eeba 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..da6e099 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -148,10 +148,19 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestRemoteExecute:
/*
- * We assume the commandTag is plain ASCII and therefore requires
- * no encoding conversion.
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
*/
- pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ if (is_tuple_shm_mq_enabled())
+ mq_putmessage_direct('C', commandTag, strlen(commandTag) + 1);
+ else
+ {
+ /*
+ * We assume the commandTag is plain ASCII and therefore requires
+ * no encoding conversion.
+ */
+ pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ }
break;
case DestNone:
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index bbad0dc..411f150 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1132,6 +1133,100 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal, NULL, 0, InvalidSnapshot);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+ (*receiver->rDestroy) (receiver);
+
+ PortalDrop(portal, false);
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d9bfa25..b8f90b7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -630,6 +630,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2445,6 +2447,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2632,6 +2644,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..784cfe0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -287,6 +287,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -497,6 +500,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
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
@@ -45,6 +45,8 @@ typedef struct ParallelContext
extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExtension(char *library_name,
char *function_name, int nworkers);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..3c56b49 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -105,4 +105,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..df56cfe
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,44 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ HeapTuple tuple;
+ int num_shm_queues;
+ bool *has_row_description;
+ bool *queue_detached;
+ bool all_queues_detached;
+ bool all_heap_fetched;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index ad7589d..067edbe 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -19,6 +19,13 @@
extern void pq_redirect_to_shm_mq(shm_mq *, shm_mq_handle *);
extern void pq_set_parallel_master(pid_t pid, BackendId backend_id);
+extern int
+mq_putmessage_direct(char msgtype, const char *s, size_t len);
+extern void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh);
+extern bool
+is_tuple_shm_mq_enabled(void);
+
extern void pq_parse_errornotice(StringInfo str, ErrorData *edata);
#endif /* PQMQ_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41288ed..86f4731 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,12 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1212,6 +1215,23 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..b6f1493 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index b1dfa85..5777271 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -23,6 +23,7 @@
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +157,15 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Oid relId;
+ List *targetList;
+ List *qual;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 316c9ce..3354398 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..576add5 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..0b6a469 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..32c3e0d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 082f7d7..ef5a320 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,9 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..91ddffe 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..8813b6d
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,30 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Oid relId, List *targetList,
+ List *qual,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 0a350fd..02cf518 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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:/messages/by-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:
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:
--
Thom
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:
/messages/by-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 patch
before applying this patch, can you try once again by first applying
the patch posted by Robert at below link:
/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
commit-id used for this patch - 0b49642
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 20 January 2015 at 16:55, Amit Kapila <amit.kapila16@gmail.com> wrote:
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 otherof
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:/messages/by-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 patch
before applying this patch, can you try once again by first applying
the patch posted by Robert at below link:/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
commit-id used for this patch - 0b49642
D'oh. Yes, you're completely right. Works fine now.
Thanks.
Thom
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:/messages/by-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'm getting an issue:
➤ 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;
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:
# 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
Note that I increased seq_page_cost to force a parallel scan in this case.
Thom
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 actually know 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 result in one worker mostly being responsible for advancing the index scan itself while the other workers were issuing (and waiting on) heap IO. So even if this doesn't turn out to be a win for seqscan, there's other places we might well want to use it.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jan 20, 2015 at 10:59 PM, Thom Brown <thom@linux.com> wrote:
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?
Yes, it should behave that way, but this is not handled in
patch as still we have to decide on what is the best execution
strategy (block-by-block or fixed chunks for different workers)
and based on that I can handle this scenario in patch.
I could return an error for such a scenario or do some work
to handle it seamlessly, but it seems to me that I have to
rework on the same if we select different approach for doing
execution than used in patch, so I am waiting for that to get
decided. I am planing to work on getting the performance data for
both the approaches, so that we can decide which is better
way to go-ahead.
The original test I performed where I was getting OOM errors now appears
to be fine:
Thanks for confirming the same.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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:
/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.comChanges 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.27 rows=8981483 width=4)
Number of Workers: 4
Number of Blocks Per Worker: 8849
(3 rows)
Though, EXPLAIN ANALYZE caused the thing.
Thanks,
Amit
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 heap
page, if it finds another tuple/tuples's from shared memory message
queue it will process those tuples first and in such a scenario, the scan
descriptor will still have reference to buffer which it is using from
scanning
from 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?
Thanks for checking the patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
<javascript:_e(%7B%7D,'cvml','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 heap
page, if it finds another tuple/tuples's from shared memory message
queue it will process those tuples first and in such a scenario, the scan
descriptor will still have reference to buffer which it is using from
scanning
from 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?
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.
Thanks,
Amit
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v5.patchapplication/octet-stream; name=parallel_seqscan_v5.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..1afac59 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -243,7 +243,19 @@ SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist, int16 *formats)
pq_sendint(&buf, 0, 2);
}
}
- pq_endmessage(&buf);
+
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
}
/*
@@ -371,7 +383,18 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
}
}
- pq_endmessage(&buf);
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..758d7e8
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,375 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+ workerResult->num_shm_queues = nWorkers;
+ workerResult->has_row_description = palloc0(sizeof(bool) * nWorkers);
+ workerResult->queue_detached = palloc0(sizeof(bool) * nWorkers);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap)
+{
+ shm_mq_result res;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ if (!resultState->all_queues_detached)
+ {
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+
+ /*
+ * Don't fetch from detached queue. This loop could continue
+ * forever, if we reach a situation such that all queue's are
+ * detached, however we won't reach here if that is the case.
+ */
+ while (resultState->queue_detached[queueId])
+ {
+ ++queueId;
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+ }
+
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, true);
+ if (res == SHM_MQ_DETACHED)
+ {
+ /*
+ * mark the queue that got detached, so that we don't
+ * try to fetch from it again.
+ */
+ resultState->queue_detached[queueId] = true;
+ resultState->has_row_description[queueId] = false;
+ --resultState->num_shm_queues;
+ /*
+ * if we have exhausted data from all worker queues, then don't
+ * process data from queues.
+ */
+ if (resultState->num_shm_queues <= 0)
+ resultState->all_queues_detached = true;
+ break;
+ }
+ else if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ bool rettuple;
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ rettuple = HandleParallelTupleMessage(resultState, tupdesc, &msg, queueId);
+ pfree(msg.data);
+ if (rettuple)
+ {
+ *fromheap = false;
+ return resultState->tuple;
+ }
+ }
+ }
+ }
+
+ /*
+ * if we have checked all the message queue's and didn't find
+ * any message or we have already fetched all the data from queue's,
+ * then it's time to fetch directly from heap. Reset the current
+ * queue as the first queue from which we need to receive tuples.
+ */
+ if ((queueId == shmScan->num_shm_queues - 1 ||
+ resultState->all_queues_detached) &&
+ !resultState->all_heap_fetched)
+ {
+ HeapTuple tuple;
+ shmScan->ss_cqueue = 0;
+ tuple = heap_getnext(scanDesc, direction);
+ if (tuple)
+ {
+ *fromheap = true;
+ return tuple;
+ }
+ else if (tuple == NULL && resultState->all_queues_detached)
+ break;
+ else
+ resultState->all_heap_fetched = true;
+ }
+ else if (resultState->all_queues_detached &&
+ resultState->all_heap_fetched)
+ break;
+
+ /* check the data in next queue. */
+ ++queueId;
+ }
+
+ return NULL;
+}
+
+/*
+ * HandleParallelTupleMessage -
+ * Handle a single tuple related protocol message received from
+ * a single parallel worker.
+ */
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ char msgtype;
+ bool rettuple = false;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description[queueId])
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description[queueId] = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(msg); /* name */
+ (void) pq_getmsgint(msg, 4); /* table OID */
+ (void) pq_getmsgint(msg, 2); /* table attnum */
+ type_id = pq_getmsgint(msg, 4); /* type OID */
+ (void) pq_getmsgint(msg, 2); /* type length */
+ (void) pq_getmsgint(msg, 4); /* typmod */
+ (void) pq_getmsgint(msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ resultState->tuple = form_result_tuple(resultState, tupdesc, msg, queueId);
+ rettuple = true;
+ break;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+ default:
+ elog(WARNING, "unknown message type: %c", msg->data[0]);
+ break;
+ }
+
+ return rettuple;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ HeapTuple tuple;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description[queueId])
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ tuple = heap_form_tuple(tupdesc, values, isnull);
+
+ /*
+ * Release locally palloc'd space. XXX would probably be good to pfree
+ * values of pass-by-reference datums, as well.
+ */
+ pfree(values);
+ pfree(isnull);
+
+ pfree(buf.data);
+
+ return tuple;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 8a0be5d..bb581a8 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -713,6 +713,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -909,6 +910,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1058,6 +1062,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1324,6 +1329,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Worker",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2141,6 +2156,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..f77a77f 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..39c624d 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,8 +191,17 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification as for them the same is done by backend
+ * worker. This case will happen only for parallel query where we push
+ * down the qualification.
+ * XXX - We can do this optimization for projection as well, but for
+ * now it is okay, as we don't allow parallel query if there are
+ * expressions involved in target list.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..1855e52
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,318 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ParallelSeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc, node->pss_currentShmScanDesc,
+ node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ direction, &fromheap);
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ fromheap ? scandesc->rs_cbuf : InvalidBuffer, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * shm_beginscan/heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+ BlockNumber end_block;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /*
+ * If we are just doing EXPLAIN (ie, aren't going to run the plan), stop
+ * here, no need to start workers.
+ */
+ if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
+ return parallelscanstate;
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(parallelscanstate->ss.ss_currentRelation->rd_id,
+ node->scan.plan.targetlist,
+ node->scan.plan.qual,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->pcxt,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+ /* Initialize the blocks to be scanned by master backend. */
+ end_block = (parallelscanstate->pcxt->nworkers + 1) *
+ node->num_blocks_per_worker;
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock =
+ end_block - node->num_blocks_per_worker;
+ /*
+ * As master backend is the last backend to scan the blocks, it
+ * should scan all the blocks.
+ */
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->endblock = InvalidBlockNumber;
+
+ /* Set the scan limits for master backend. */
+ heap_setscanlimits(parallelscanstate->ss.ss_currentScanDesc,
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock,
+ (parallelscanstate->ss.ss_currentScanDesc->rs_nblocks -
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock));
+
+ /*
+ * Use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * Number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(parallelscanstate->pcxt->nworkers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ parallelscanstate->pcxt->nworkers);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt)
+ {
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 3cb81fc..5780df0 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,22 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index f12f2d5..cfab8b5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -26,6 +26,8 @@ static bool pq_mq_busy = false;
static pid_t pq_mq_parallel_master_pid = 0;
static pid_t pq_mq_parallel_master_backend_id = InvalidBackendId;
+static shm_mq_handle *pq_mq_tuple_handle = NULL;
+
static void mq_comm_reset(void);
static int mq_flush(void);
static int mq_flush_if_writable(void);
@@ -61,6 +63,26 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
}
/*
+ * Arrange to send some frontend/backend protocol messages to a shared-memory
+ * tuple message queue.
+ */
+void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh)
+{
+ pq_mq_tuple_handle = mqh;
+}
+
+/*
+ * Check if tuples can be sent through tuple shared-memory
+ * message queue.
+ */
+bool
+is_tuple_shm_mq_enabled(void)
+{
+ return pq_mq_tuple_handle ? true : false;
+}
+
+/*
* Arrange to SendProcSignal() to the parallel master each time we transmit
* message data via the shm_mq.
*/
@@ -161,6 +183,42 @@ mq_putmessage(char msgtype, const char *s, size_t len)
return 0;
}
+/*
+ * Transmit a libpq protocol message to the shared memory message queue
+ * via pq_mq_tuple_handle. We don't include a length word, because the
+ * receiver will know the length of the message from shm_mq_receive().
+ */
+int
+mq_putmessage_direct(char msgtype, const char *s, size_t len)
+{
+ shm_mq_iovec iov[2];
+ shm_mq_result result;
+
+ iov[0].data = &msgtype;
+ iov[0].len = 1;
+ iov[1].data = s;
+ iov[1].len = len;
+
+ Assert(pq_mq_tuple_handle != NULL);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_tuple_handle, iov, 2, true);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ Assert(result == SHM_MQ_SUCCESS || result == SHM_MQ_DETACHED);
+ if (result != SHM_MQ_SUCCESS)
+ return EOF;
+ return 0;
+}
+
static void
mq_putmessage_noblock(char msgtype, const char *s, size_t len)
{
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 020558b..4abfd25 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +227,73 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..5245652
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,126 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 655be81..1c7f640 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -1133,6 +1147,71 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock)
+{
+ SeqScan *scan_plan;
+
+ /*
+ * Pass scan_relid as 1, this is okay for now as sequence scan worker
+ * is allowed to operate on just one relation.
+ * XXX - we should ideally get scanrelid from master backend.
+ */
+ scan_plan = make_seqscan(targetList,
+ scan_clauses,
+ 1);
+
+ scan_plan->startblock = startBlock;
+ scan_plan->endblock = endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3314,6 +3393,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9cbbcfb..d2b1621 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,71 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ AclMode required_access = ACL_SELECT;
+ RangeTblEntry *rte;
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ rte = makeNode(RangeTblEntry);
+ rte->rtekind = RTE_RELATION;
+ rte->relid = workerstmt->relId;
+ rte->relkind = 'r';
+ rte->requiredPerms = required_access;
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) workerstmt->qual);
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, workerstmt->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ scan_plan = create_worker_seqscan_plan(workerstmt->targetList,
+ workerstmt->qual,
+ workerstmt->startBlock,
+ workerstmt->endBlock);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = list_make1(rte);
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, rte->relid);;
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 7703946..3a44aef 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..538e612 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,41 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks. We always consider master backend
+ * as last worker because it will first try to get the tuples
+ * scanned by other workers. For calculation of number of blocks
+ * per worker, an additional worker needs to be consider for
+ * master backend.
+ */
+ pathnode->num_blocks_per_worker = rel->pages / (nWorkers + 1);
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..d52d1b6
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,224 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PG_WORKER_KEY_RELID 0
+#define PG_WORKER_KEY_TARGETLIST 1
+#define PG_WORKER_KEY_QUAL 2
+#define PG_WORKER_KEY_BLOCKS 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+
+static void exec_worker_message(dsm_segment *seg, shm_toc *toc);
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Oid relId, List *targetList, List *qual,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ int i;
+ Size targetlist_len, qual_len;
+ BlockNumber *num_blocks_per_worker;
+ Oid *reliddata;
+ char *targetlistdata;
+ char *targetlist_str;
+ char *qualdata;
+ char *qual_str;
+ char *tuple_queue_space;
+ ParallelContext *pcxt;
+ shm_mq *mq;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(exec_worker_message, nWorkers);
+
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(relId));
+
+ targetlist_str = nodeToString(targetList);
+ targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, targetlist_len);
+
+ qual_str = nodeToString(qual);
+ qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, qual_len);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(BlockNumber));
+
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * nWorkers);
+
+ /* 5 keys for parallel seq. scan specific data. */
+ shm_toc_estimate_keys(&pcxt->estimator, 5);
+
+ InitializeParallelDSM(pcxt);
+
+ /* Store scan relation id in dynamic shared memory. */
+ reliddata = shm_toc_allocate(pcxt->toc, sizeof(Oid));
+ *reliddata = relId;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_RELID, reliddata);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(pcxt->toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(pcxt->toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_QUAL, qualdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(pcxt->toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(pcxt->toc, PG_WORKER_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(pcxt);
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], pcxt->worker[i].bgwhandle);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+
+/*
+ * exec_worker_message
+ *
+ * Execute the work assigned to a worker by master backend.
+ */
+void
+exec_worker_message(dsm_segment *seg, shm_toc *toc)
+{
+ char *targetlistdata;
+ char *qualdata;
+ char *tuple_queue_space;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ Oid *relId;
+ List *targetList = NIL;
+ List *qual = NIL;
+ worker_stmt *workerstmt;
+
+ relId = shm_toc_lookup(toc, PG_WORKER_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PG_WORKER_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PG_WORKER_KEY_QUAL);
+ num_blocks_per_worker = shm_toc_lookup(toc, PG_WORKER_KEY_BLOCKS);
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = (ParallelWorkerNumber + 1) * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_tuple_shm_mq(responseq);
+
+ /* Restore targetList and qual passed by main backend. */
+ targetList = (List *) stringToNode(targetlistdata);
+ qual = (List *) stringToNode(qualdata);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->relId = *relId;
+ workerstmt->targetList = targetList;
+ workerstmt->qual = qual;
+ workerstmt->startBlock = start_block;
+
+ /*
+ * Last worker should scan all the remaining blocks.
+ *
+ * XXX - It is possible that expected number of workers
+ * won't get started, so to handle such cases master
+ * backend should scan remaining blocks.
+ */
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 47ed84c..994eeba 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..da6e099 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -148,10 +148,19 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestRemoteExecute:
/*
- * We assume the commandTag is plain ASCII and therefore requires
- * no encoding conversion.
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
*/
- pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ if (is_tuple_shm_mq_enabled())
+ mq_putmessage_direct('C', commandTag, strlen(commandTag) + 1);
+ else
+ {
+ /*
+ * We assume the commandTag is plain ASCII and therefore requires
+ * no encoding conversion.
+ */
+ pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ }
break;
case DestNone:
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index bbad0dc..411f150 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1132,6 +1133,100 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal, NULL, 0, InvalidSnapshot);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+ (*receiver->rDestroy) (receiver);
+
+ PortalDrop(portal, false);
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d9bfa25..b8f90b7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -630,6 +630,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2445,6 +2447,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2632,6 +2644,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..784cfe0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -287,6 +287,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -497,6 +500,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
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
@@ -45,6 +45,8 @@ typedef struct ParallelContext
extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExtension(char *library_name,
char *function_name, int nworkers);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..3c56b49 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -105,4 +105,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..df56cfe
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,44 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ HeapTuple tuple;
+ int num_shm_queues;
+ bool *has_row_description;
+ bool *queue_detached;
+ bool all_queues_detached;
+ bool all_heap_fetched;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index ad7589d..067edbe 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -19,6 +19,13 @@
extern void pq_redirect_to_shm_mq(shm_mq *, shm_mq_handle *);
extern void pq_set_parallel_master(pid_t pid, BackendId backend_id);
+extern int
+mq_putmessage_direct(char msgtype, const char *s, size_t len);
+extern void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh);
+extern bool
+is_tuple_shm_mq_enabled(void);
+
extern void pq_parse_errornotice(StringInfo str, ErrorData *edata);
#endif /* PQMQ_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41288ed..86f4731 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,12 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1212,6 +1215,23 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..b6f1493 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index b1dfa85..5777271 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -23,6 +23,7 @@
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +157,15 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Oid relId;
+ List *targetList;
+ List *qual;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 316c9ce..3354398 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..576add5 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..0b6a469 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..32c3e0d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 082f7d7..ef5a320 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,9 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(List *targetList, List *scan_clauses,
+ BlockNumber startBlock, BlockNumber endBlock);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..91ddffe 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..8813b6d
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,30 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Oid relId, List *targetList,
+ List *qual,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 0a350fd..02cf518 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 on test (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: 9960972
Planning time: 0.206 ms
Execution 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 Seq Scan 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: 8849
Planning time: 0.137 ms
Execution 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Import Notes
Reply to msg id not found: CAA4eK1Jf8Bxt2BHL-o-xJqK0RJn75_yrs9NoEKALRYYMaJ9Tng@mail.gmail.com
On Thu, Jan 22, 2015 at 10:44 AM, Amit Langote <
Langote_Amit_f8@lab.ntt.co.jp> wrote:
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.
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.
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.
By the way, 17s and 6s compare really well in favor of parallel seqscan
above, :)
That sounds interesting.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Jan 19, 2015 at 6:50 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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,
I have used parallel_count module which is part of parallel-mode patch
as that seems to be more close to verify the I/O pattern (doesn't have any
tuple communication overhead).
Script used to test is attached (parallel_count.sh)
Performance Data
----------------------------
Configuration and Db Details
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');
*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
Then I have modified the parallel_count module such that it can scan in
fixed chunks, rather than block-by-block, the patch for same is attached
(parallel_count_fixed_chunk_v1.patch, this is a patch based on parallel
count module in parallel-mode patch [1]/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com).
*Fixed-Chunks*
*No. of workers/Time (ms)* *0* *2*
286346 234037
250051 215111
255915 254934
263754 242228
251399 202581
Observations
------------------------
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.
Now I can go-ahead and try with prefetching approach as suggested
by you, but I have a feeling that overall it might not be beneficial (mainly
due to the reason that it is not supported on all platforms, we can say
that we don't care for such platforms, but still there is no mitigation
strategy
for those platforms) due to the reasons mentioned up-thread.
Thoughts?
[1]: /messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_count_fixed_chunk_v1.patchapplication/octet-stream; name=parallel_count_fixed_chunk_v1.patchDownload
diff --git a/contrib/parallel_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
index 0a32ea8..0b75694 100644
--- a/contrib/parallel_dummy/parallel_dummy.c
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -43,8 +43,13 @@ typedef struct
BlockNumber lastblock;
BlockNumber currentblock;
int64 ntuples;
+ int workers_attached;
+ int workers_expected;
+ BlockNumber num_blocks_per_worker;
} ParallelCountInfo;
+int ParallelWorkerNumber;
+
void _PG_init(void);
void sleep_worker_main(dsm_segment *seg, shm_toc *toc);
void count_worker_main(dsm_segment *seg, shm_toc *toc);
@@ -122,6 +127,9 @@ parallel_count(PG_FUNCTION_ARGS)
info->lastblock = RelationGetNumberOfBlocks(rel);
info->currentblock = 0;
info->ntuples = 0;
+ info->workers_attached = 0;
+ info->workers_expected = nworkers;
+ info->num_blocks_per_worker = info->lastblock / (nworkers + 1);
shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
LaunchParallelWorkers(pcxt);
@@ -175,32 +183,35 @@ count_helper(Relation rel, ParallelCountInfo *info)
int64 mytuples = 0;
Oid relid = info->relid;
Snapshot snapshot = GetActiveSnapshot();
-
- for (;;)
+ BlockNumber blkno;
+ BlockNumber end_block;
+ BlockNumber start_block;
+ Buffer buffer;
+ Page page;
+ int lines;
+ OffsetNumber lineoff;
+ ItemId lpp;
+ bool all_visible;
+
+
+ SpinLockAcquire(&info->mutex);
+ ParallelWorkerNumber = info->workers_attached++;
+ SpinLockRelease(&info->mutex);
+
+ end_block = (ParallelWorkerNumber + 1) * info->num_blocks_per_worker;
+ start_block = end_block - info->num_blocks_per_worker;
+
+ /*
+ * Last worker is responsible for scanning all the remaining
+ * blocks in relation.
+ */
+ if (ParallelWorkerNumber == info->workers_expected)
+ end_block = info->lastblock;
+
+ for (blkno = start_block; blkno < end_block; blkno++)
{
- BlockNumber blkno;
- Buffer buffer;
- Page page;
- int lines;
- OffsetNumber lineoff;
- ItemId lpp;
- bool all_visible;
- bool done = false;
-
CHECK_FOR_INTERRUPTS();
- SpinLockAcquire(&info->mutex);
- if (info->currentblock >= info->lastblock)
- done = true;
- else
- blkno = info->currentblock++;
- info->ntuples += ntuples;
- SpinLockRelease(&info->mutex);
-
- mytuples += ntuples;
- if (done)
- break;
-
buffer = ReadBuffer(rel, blkno);
LockBuffer(buffer, BUFFER_LOCK_SHARE);
page = BufferGetPage(buffer);
@@ -210,8 +221,8 @@ count_helper(Relation rel, ParallelCountInfo *info)
all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
- lineoff <= lines;
- lineoff++, lpp++)
+ lineoff <= lines;
+ lineoff++, lpp++)
{
HeapTupleData loctup;
@@ -232,6 +243,12 @@ count_helper(Relation rel, ParallelCountInfo *info)
}
UnlockReleaseBuffer(buffer);
+
+ SpinLockAcquire(&info->mutex);
+ info->ntuples += ntuples;
+ SpinLockRelease(&info->mutex);
+
+ mytuples += ntuples;
}
elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Jan 22, 2015 at 7:23 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
Import Notes
Reply to msg id not found: WM5099753816f39831f93e2c406c153079dda8fdb7b4134fb0aa1033d1e19fa7dda29bebfd99bbb46513d57c61910a0055@asav-1.01.com
On Thu, Jan 22, 2015 at 7:23 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
Below is the data with more number of workers, the amount of data and
other configurations remains as previous, I have only increased parallel
worker count:
*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
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 39766The 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 to 1/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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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."
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Jan 24, 2015 at 12:24 AM, Joshua D. Drake <jd@commandprompt.com>
wrote:
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?
There is variation in tests at different worker count but there is
definitely improvement from 0 to 2 worker count (if you refer my
initial mail on this data, with 2 workers there is a benefit of ~20%)
and I think we run the tests in a similar way (like compare 0 and 2
or 0 or 4 or 0 and 8), then the other effects could be minimised and
we might see better consistency, however the general trend with
fixed-chunk seems to be that scanning that way is better.
I think the real benefit with the current approach/patch can be seen
with qualifications (especially costly expression evaluation).
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.
cached? First couple of runs gets the relations into memory?
Not entirely, as the table size is double than RAM, so each run
has to perform I/O.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 because complex 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 certainly falls 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 them in a queue (or shove a pointer to a buffer in the queue). Similarly, have IndexScans have one worker reading the index and 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 because complex 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 certainly falls 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jan 27, 2015 at 3:18 AM, Jim Nasby <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 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
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 = 64GBTable 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 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
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 sensitivity to IO latency). Perhaps the way to do that is with the parallel infrastructure, perhaps not. But I think it's premature to look at parallelism for increasing IO performance, or worrying about things like how many IO threads we should have before we at least look at simpler things we could do. We shouldn't assume there's nothing to be gained short of 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 performance I could get with dd. Once I got up to 4-8 SeqScans of different tables running together, I saw iostat numbers that were similar to what a single dd bs=8k would do. I've tested this with iSCSI SAN volumes on both 1Gbit and 10Gbit ethernet.
This is why I think that when it comes to IO performance, before we start worrying about real parallelization we should investigate ways 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 call into 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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]Not my idea.. 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.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 28, 2015 at 12:38 PM, Heikki Linnakangas <
hlinnakangas@vmware.com> wrote:
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
will get any major benefit via parallel sequential scan unless
the qualification expressions or other expressions used in
statement are costly. So here one way could be that either user
should configure the parallel sequence scan parameters in such
a way that only when it can be beneficial it should perform parallel
scan (something like increase parallel_tuple_comm_cost or we can
have some another parameter) or just not use parallel sequential scan
(parallel_seqscan_degree=0).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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?
Thom
On Wed, Jan 28, 2015 at 7:46 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
* 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 suspect that'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 requesting stuff 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 entirely premature.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 suspect that'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 requesting stuff 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 entirely premature.
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jan 28, 2015 at 8:59 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
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
either between block-by-block or fixed_chuck approach, the data is at
end of mail for your reference. There is variation in some cases like in
fixed_chunk approach, in 8 workers case it is showing lesser time, however
on certain executions it has taken almost the same time as other workers.
Now if we go with block-by-block approach then we have advantage that
the work distribution granularity will be smaller and hence better and if
we go with chunk-by-chunk (fixed_chunk of 1GB) approach, then there
is good chance that kernel can do the better optimization for reading it.
Based on inputs on this thread, one way for execution strategy could
be:
a. In optimizer, based on effective_io_concurrency, size of relation and
parallel_seqscan_degree, we can decide how many workers can be
used for executing the plan
- choose the number_of_workers equal to effective_io_concurrency,
if it is less than parallel_seqscan_degree, else number_of_workers
will be equal to parallel_seqscan_degree.
- if the size of relation is greater than number_of_workers times GB
(if number_of_workers is 8, then we need to compare the size of
relation with 8GB), then keep number_of_workers intact and distribute
the remaining chunks/segments during execution, else
reduce the number_of_workers such that each worker gets 1GB
to operate.
- if the size of relation is less than 1GB, then we can either not
choose the parallel_seqscan at all or could use smaller chunks
or could use block-by-block approach to execute.
- here we need to consider other parameters like parallel_setup
parallel_startup and tuple_communication cost as well.
b. In executor, if less workers are available than what are required
for statement execution, then we can redistribute the remaining
work among workers.
Performance Data - Before first run of each worker, I have executed
drop_caches to clear the cache and restarted the server, so we can
assume that except Run-1, all other runs have some caching effect.
*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
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Jan 27, 2015 at 11:08 PM, Heikki Linnakangas <
hlinnakangas@vmware.com> wrote:
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?
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?
Cheers,
Jeff
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 suspect that'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 was an 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's just 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 doing read-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 explicitly get that behavior where we want it.
--
Jim Nasby, Data Architect, Blue Treble Consulting
Data in Trouble? Get it in Treble! http://BlueTreble.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
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
Attachments:
0001-Quick-proof-of-concept-for-indexscan-prefetching.patchtext/x-diffDownload
>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
0002-Fix-index-only-scan-and-rescan.patchtext/x-diffDownload
>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
0003-First-try-on-tuple-look-ahead-in-nestloop.patchtext/x-diffDownload
>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
0004-Limit-recursive-prefetching-for-merge-join.patchtext/x-diffDownload
>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 10:30 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
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
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.
Note -
a. it is still based on parallel-mode-v1 [1]/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com patch of Robert.
b. based on CommitId - fd496129 [on top of this commit, apply
Robert's patch and then the attached patch]
c. just build and tested on Windows, my linux box has some
problem, will fix that soon and verify this on linux as well.
[1]: /messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
/messages/by-id/CA+TgmoZdUK4K3XHBxc9vM-82khourEZdvQWTfgLhWsd2R2aAGQ@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v6.patchapplication/octet-stream; name=parallel_seqscan_v6.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..1afac59 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -243,7 +243,19 @@ SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist, int16 *formats)
pq_sendint(&buf, 0, 2);
}
}
- pq_endmessage(&buf);
+
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
}
/*
@@ -371,7 +383,18 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
}
}
- pq_endmessage(&buf);
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 57408d3..784d79d 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -307,6 +307,12 @@ heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk, BlockNumber numBlks)
scan->rs_numblocks = numBlks;
}
+void
+heap_setsyncscan(HeapScanDesc scan, bool sync_scan)
+{
+ scan->rs_syncscan = sync_scan;
+}
+
/*
* heapgetpage - subroutine for heapgettup()
*
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..758d7e8
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,375 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+
+/*
+ * shm_beginscan -
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult -
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+ workerResult->num_shm_queues = nWorkers;
+ workerResult->has_row_description = palloc0(sizeof(bool) * nWorkers);
+ workerResult->queue_detached = palloc0(sizeof(bool) * nWorkers);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext -
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap)
+{
+ shm_mq_result res;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ if (!resultState->all_queues_detached)
+ {
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+
+ /*
+ * Don't fetch from detached queue. This loop could continue
+ * forever, if we reach a situation such that all queue's are
+ * detached, however we won't reach here if that is the case.
+ */
+ while (resultState->queue_detached[queueId])
+ {
+ ++queueId;
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+ }
+
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, true);
+ if (res == SHM_MQ_DETACHED)
+ {
+ /*
+ * mark the queue that got detached, so that we don't
+ * try to fetch from it again.
+ */
+ resultState->queue_detached[queueId] = true;
+ resultState->has_row_description[queueId] = false;
+ --resultState->num_shm_queues;
+ /*
+ * if we have exhausted data from all worker queues, then don't
+ * process data from queues.
+ */
+ if (resultState->num_shm_queues <= 0)
+ resultState->all_queues_detached = true;
+ break;
+ }
+ else if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ bool rettuple;
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ rettuple = HandleParallelTupleMessage(resultState, tupdesc, &msg, queueId);
+ pfree(msg.data);
+ if (rettuple)
+ {
+ *fromheap = false;
+ return resultState->tuple;
+ }
+ }
+ }
+ }
+
+ /*
+ * if we have checked all the message queue's and didn't find
+ * any message or we have already fetched all the data from queue's,
+ * then it's time to fetch directly from heap. Reset the current
+ * queue as the first queue from which we need to receive tuples.
+ */
+ if ((queueId == shmScan->num_shm_queues - 1 ||
+ resultState->all_queues_detached) &&
+ !resultState->all_heap_fetched)
+ {
+ HeapTuple tuple;
+ shmScan->ss_cqueue = 0;
+ tuple = heap_getnext(scanDesc, direction);
+ if (tuple)
+ {
+ *fromheap = true;
+ return tuple;
+ }
+ else if (tuple == NULL && resultState->all_queues_detached)
+ break;
+ else
+ resultState->all_heap_fetched = true;
+ }
+ else if (resultState->all_queues_detached &&
+ resultState->all_heap_fetched)
+ break;
+
+ /* check the data in next queue. */
+ ++queueId;
+ }
+
+ return NULL;
+}
+
+/*
+ * HandleParallelTupleMessage -
+ * Handle a single tuple related protocol message received from
+ * a single parallel worker.
+ */
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ char msgtype;
+ bool rettuple = false;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'T':
+ {
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+
+ if (resultState->has_row_description[queueId])
+ elog(ERROR, "multiple RowDescription messages");
+ resultState->has_row_description[queueId] = true;
+ if (natts != tupdesc->natts)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("worker result rowtype does not match "
+ "the specified FROM clause rowtype")));
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid type_id;
+
+ (void) pq_getmsgstring(msg); /* name */
+ (void) pq_getmsgint(msg, 4); /* table OID */
+ (void) pq_getmsgint(msg, 2); /* table attnum */
+ type_id = pq_getmsgint(msg, 4); /* type OID */
+ (void) pq_getmsgint(msg, 2); /* type length */
+ (void) pq_getmsgint(msg, 4); /* typmod */
+ (void) pq_getmsgint(msg, 2); /* format code */
+
+ if (type_id != tupdesc->attrs[i]->atttypid)
+ ereport(ERROR,
+ (errcode(ERRCODE_DATATYPE_MISMATCH),
+ errmsg("remote query result rowtype does not match "
+ "the specified FROM clause rowtype")));
+ }
+
+ pq_getmsgend(msg);
+
+ break;
+ }
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ resultState->tuple = form_result_tuple(resultState, tupdesc, msg, queueId);
+ rettuple = true;
+ break;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+ default:
+ elog(WARNING, "unknown message type: %c", msg->data[0]);
+ break;
+ }
+
+ return rettuple;
+}
+
+/*
+ * form_result_tuple -
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ HeapTuple tuple;
+ StringInfoData buf;
+
+ if (!resultState->has_row_description[queueId])
+ elog(ERROR, "DataRow not preceded by RowDescription");
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ tuple = heap_form_tuple(tupdesc, values, isnull);
+
+ /*
+ * Release locally palloc'd space. XXX would probably be good to pfree
+ * values of pass-by-reference datums, as well.
+ */
+ pfree(values);
+ pfree(isnull);
+
+ pfree(buf.data);
+
+ return tuple;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 7cfc9bb..3b5b4c6 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -917,6 +918,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1066,6 +1070,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1207,6 +1212,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (nodeTag(plan) == T_ParallelSeqScan)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((ParallelSeqScanState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((ParallelSeqScanState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1332,6 +1355,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ ExplainPropertyInteger("Number of Blocks Per Worker",
+ ((ParallelSeqScan *) plan)->num_blocks_per_worker, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2224,6 +2257,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/commands/portalcmds.c b/src/backend/commands/portalcmds.c
index 2794537..33eef6e 100644
--- a/src/backend/commands/portalcmds.c
+++ b/src/backend/commands/portalcmds.c
@@ -121,7 +121,7 @@ PerformCursorOpen(PlannedStmt *stmt, ParamListInfo params,
/*
* Start execution, inserting parameters if any.
*/
- PortalStart(portal, params, 0, GetActiveSnapshot());
+ PortalStart(portal, params, 0, GetActiveSnapshot(), 0);
Assert(portal->strategy == PORTAL_ONE_SELECT);
diff --git a/src/backend/commands/prepare.c b/src/backend/commands/prepare.c
index 71b08f0..93ae6b3 100644
--- a/src/backend/commands/prepare.c
+++ b/src/backend/commands/prepare.c
@@ -289,7 +289,7 @@ ExecuteQuery(ExecuteStmt *stmt, IntoClause *intoClause,
/*
* Run the portal as appropriate.
*/
- PortalStart(portal, paramLI, eflags, GetActiveSnapshot());
+ PortalStart(portal, paramLI, eflags, GetActiveSnapshot(), 0);
(void) PortalRun(portal, count, false, dest, dest, completionTag);
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 6414cb9..858e5e8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -332,7 +332,29 @@ standard_ExecutorRun(QueryDesc *queryDesc,
(*dest->rShutdown) (dest);
if (queryDesc->totaltime)
+ {
InstrStopNode(queryDesc->totaltime, estate->es_processed);
+ /*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ /*if (nodeTag(queryDesc->planstate->plan) == T_ParallelSeqScan)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers =
+ ((ParallelSeqScanState *)queryDesc->planstate)->pcxt->nworkers;
+ char *inst_info_workers =
+ ((ParallelSeqScanState *)queryDesc->planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker =
+ (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(queryDesc->planstate->instrument, instrument_worker);
+ }
+ }*/
+ }
MemoryContextSwitchTo(oldcontext);
}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..f77a77f 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..39c624d 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,8 +191,17 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification as for them the same is done by backend
+ * worker. This case will happen only for parallel query where we push
+ * down the qualification.
+ * XXX - We can do this optimization for projection as well, but for
+ * now it is okay, as we don't allow parallel query if there are
+ * expressions involved in target list.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..b7898a5 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -21,6 +21,8 @@ BufferUsage pgBufferUsage;
static void BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add, const BufferUsage *sub);
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
/* Allocate new instrumentation structure(s) */
@@ -127,6 +129,28 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
static void
BufferUsageAccumDiff(BufferUsage *dst,
@@ -148,3 +172,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
\ No newline at end of file
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..b7a9e79
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,329 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan sequentially scans a relation.
+ * ExecSeqNext retrieve next tuple in sequential order.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ParallelSeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc, node->pss_currentShmScanDesc,
+ node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ direction, &fromheap);
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ fromheap ? scandesc->rs_cbuf : InvalidBuffer, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * shm_beginscan/heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(SeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+
+ /*
+ * Each backend worker participating in parallel sequiantial
+ * scan operate on different set of blocks, so there doesn't
+ * seem to much benefit in allowing sync scans.
+ */
+ heap_setsyncscan(currentScanDesc, false);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+ BlockNumber end_block;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitParallelScanRelation(¶llelscanstate->ss, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /*
+ * If we are just doing EXPLAIN (ie, aren't going to run the plan), stop
+ * here, no need to start workers.
+ */
+ if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
+ return parallelscanstate;
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitiateWorkers(((SeqScan *) parallelscanstate->ss.ps.plan)->scanrelid,
+ node->scan.plan.targetlist,
+ node->scan.plan.qual,
+ estate->es_range_table,
+ estate->es_param_list_info,
+ estate->es_instrument,
+ ¶llelscanstate->inst_options_space,
+ ¶llelscanstate->responseq,
+ ¶llelscanstate->pcxt,
+ node->num_blocks_per_worker,
+ node->num_workers);
+
+ /* Initialize the blocks to be scanned by master backend. */
+ end_block = (parallelscanstate->pcxt->nworkers + 1) *
+ node->num_blocks_per_worker;
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock =
+ end_block - node->num_blocks_per_worker;
+ /*
+ * As master backend is the last backend to scan the blocks, it
+ * should scan all the blocks.
+ */
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->endblock = InvalidBlockNumber;
+
+ /* Set the scan limits for master backend. */
+ heap_setscanlimits(parallelscanstate->ss.ss_currentScanDesc,
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock,
+ (parallelscanstate->ss.ss_currentScanDesc->rs_nblocks -
+ ((SeqScan*) parallelscanstate->ss.ps.plan)->startblock));
+
+ /*
+ * Use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backends would have put the data after projection.
+ * Number of queue's must be equal to number of worker backends.
+ */
+ currentShmScanDesc = shm_beginscan(parallelscanstate->pcxt->nworkers);
+ workerResult = ExecInitWorkerResult(parallelscanstate->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ parallelscanstate->pcxt->nworkers);
+
+ parallelscanstate->pss_currentShmScanDesc = currentShmScanDesc;
+ parallelscanstate->pss_workerResult = workerResult;
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation sequentially from multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt)
+ {
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
diff --git a/src/backend/executor/nodeSeqscan.c b/src/backend/executor/nodeSeqscan.c
index 3cb81fc..5107950 100644
--- a/src/backend/executor/nodeSeqscan.c
+++ b/src/backend/executor/nodeSeqscan.c
@@ -139,6 +139,40 @@ InitScanRelation(SeqScanState *node, EState *estate, int eflags)
0,
NULL);
+ /*
+ * set the scan limits, if requested by plan. If the end block
+ * is not specified, then scan all the blocks till end.
+ */
+ if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber &&
+ ((SeqScan *) node->ps.plan)->endblock != InvalidBlockNumber)
+ {
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (((SeqScan *) node->ps.plan)->endblock -
+ ((SeqScan *) node->ps.plan)->startblock));
+
+ /*
+ * Each backend worker participating in parallel sequiantial
+ * scan operate on different set of blocks, so there doesn't
+ * seem to much benefit in allowing sync scans.
+ */
+ heap_setsyncscan(currentScanDesc, false);
+ }
+ else if (((SeqScan *) node->ps.plan)->startblock != InvalidBlockNumber)
+ {
+ heap_setscanlimits(currentScanDesc,
+ ((SeqScan *) node->ps.plan)->startblock,
+ (currentScanDesc->rs_nblocks -
+ ((SeqScan *) node->ps.plan)->startblock));
+
+ /*
+ * Each backend worker participating in parallel sequiantial
+ * scan operate on different set of blocks, so there doesn't
+ * seem to much benefit in allowing sync scans.
+ */
+ heap_setsyncscan(currentScanDesc, false);
+ }
+
node->ss_currentRelation = currentRelation;
node->ss_currentScanDesc = currentScanDesc;
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 3a93a04..f7da680 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -1376,7 +1376,7 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
/*
* Start portal execution.
*/
- PortalStart(portal, paramLI, 0, snapshot);
+ PortalStart(portal, paramLI, 0, snapshot, 0);
Assert(portal->strategy != PORTAL_MULTI_QUERY);
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index f12f2d5..cfab8b5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -26,6 +26,8 @@ static bool pq_mq_busy = false;
static pid_t pq_mq_parallel_master_pid = 0;
static pid_t pq_mq_parallel_master_backend_id = InvalidBackendId;
+static shm_mq_handle *pq_mq_tuple_handle = NULL;
+
static void mq_comm_reset(void);
static int mq_flush(void);
static int mq_flush_if_writable(void);
@@ -61,6 +63,26 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
}
/*
+ * Arrange to send some frontend/backend protocol messages to a shared-memory
+ * tuple message queue.
+ */
+void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh)
+{
+ pq_mq_tuple_handle = mqh;
+}
+
+/*
+ * Check if tuples can be sent through tuple shared-memory
+ * message queue.
+ */
+bool
+is_tuple_shm_mq_enabled(void)
+{
+ return pq_mq_tuple_handle ? true : false;
+}
+
+/*
* Arrange to SendProcSignal() to the parallel master each time we transmit
* message data via the shm_mq.
*/
@@ -161,6 +183,42 @@ mq_putmessage(char msgtype, const char *s, size_t len)
return 0;
}
+/*
+ * Transmit a libpq protocol message to the shared memory message queue
+ * via pq_mq_tuple_handle. We don't include a length word, because the
+ * receiver will know the length of the message from shm_mq_receive().
+ */
+int
+mq_putmessage_direct(char msgtype, const char *s, size_t len)
+{
+ shm_mq_iovec iov[2];
+ shm_mq_result result;
+
+ iov[0].data = &msgtype;
+ iov[0].len = 1;
+ iov[1].data = s;
+ iov[1].len = len;
+
+ Assert(pq_mq_tuple_handle != NULL);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_tuple_handle, iov, 2, true);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ Assert(result == SHM_MQ_SUCCESS || result == SHM_MQ_DETACHED);
+ if (result != SHM_MQ_SUCCESS)
+ return EOF;
+ return 0;
+}
+
static void
mq_putmessage_noblock(char msgtype, const char *s, size_t len)
{
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1a24f5..b1e1d19 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -317,6 +317,8 @@ CopyScanFields(const Scan *from, Scan *newnode)
CopyPlanFields((const Plan *) from, (Plan *) newnode);
COPY_SCALAR_FIELD(scanrelid);
+ COPY_SCALAR_FIELD(startblock);
+ COPY_SCALAR_FIELD(endblock);
}
/*
@@ -352,6 +354,28 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyParallelSeqScan
+ */
+static ParallelSeqScan *
+_copyParallelSeqScan(const ParallelSeqScan *from)
+{
+ ParallelSeqScan *newnode = makeNode(ParallelSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+ COPY_SCALAR_FIELD(num_blocks_per_worker);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4039,6 +4063,9 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_ParallelSeqScan:
+ retval = _copyParallelSeqScan(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index dd1278b..0b9c969 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -285,6 +285,8 @@ _outScanInfo(StringInfo str, const Scan *node)
_outPlanInfo(str, (const Plan *) node);
WRITE_UINT_FIELD(scanrelid);
+ WRITE_UINT_FIELD(startblock);
+ WRITE_UINT_FIELD(endblock);
}
/*
@@ -437,6 +439,17 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outParallelSeqScan(StringInfo str, const ParallelSeqScan *node)
+{
+ WRITE_NODE_TYPE("PARALLELSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+ WRITE_UINT_FIELD(num_blocks_per_worker);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2851,6 +2864,9 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_ParallelSeqScan:
+ _outParallelSeqScan(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index 2f2f5ed..b56f6c7 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -74,3 +87,185 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+ /* sizeof(ParamListInfoData) includes the first array element */
+ size = sizeof(ParamListInfoData) +
+ (num_params - 1) * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 020558b..4abfd25 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +227,73 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..fda6f40
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,148 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 655be81..8abad5e 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ BlockNumber nblocksperworker);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +560,7 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1148,65 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ * Returns a seqscan plan for the base relation scanned by worker
+ * with restriction clauses 'qual' and targetlist 'tlist'.
+ */
+SeqScan *
+create_worker_seqscan_plan(worker_stmt *workerstmt)
+{
+ SeqScan *scan_plan;
+
+ scan_plan = make_seqscan(workerstmt->targetList,
+ workerstmt->qual,
+ workerstmt->scanrelId);
+
+ scan_plan->startblock = workerstmt->startBlock;
+ scan_plan->endblock = workerstmt->endBlock;
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ * Returns a seqscan plan for the base relation scanned by 'best_path'
+ * with restriction clauses 'scan_clauses' and targetlist 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ best_path->num_blocks_per_worker);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3314,6 +3388,30 @@ make_seqscan(List *qptlist,
plan->lefttree = NULL;
plan->righttree = NULL;
node->scanrelid = scanrelid;
+ node->startblock = InvalidBlockNumber;
+ node->endblock = InvalidBlockNumber;
+
+ return node;
+}
+
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ BlockNumber nblocksperworker)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->num_blocks_per_worker = nblocksperworker;
return node;
}
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9cbbcfb..4f8e4d3 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,69 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ * Returns a planned statement to be used by worker for execution.
+ * Ideally, master backend should form worker's planned statement
+ * and pass the same to worker, however for now master backend
+ * just passes the required information and PlannedStmt is then
+ * constructed by worker.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt)
+{
+ SeqScan *scan_plan;
+ PlannedStmt *result;
+ ListCell *tlist;
+ Oid reloid;
+
+ /* get the relid to save the same as part of planned statement. */
+ reloid = getrelid(workerstmt->scanrelId, workerstmt->rangetableList);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) workerstmt->qual);
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, workerstmt->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ scan_plan = create_worker_seqscan_plan(workerstmt);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = workerstmt->rangetableList;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, reloid);
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to get hasRowSecurity passed from master
+ * backend as this is used only for invalidation and in
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 7703946..3a44aef 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 78fb6b1..c35f934 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2163,6 +2163,7 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_ParallelSeqScan:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..538e612 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,41 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+ /*
+ * Divide the work equally among all the workers, for cases
+ * where division is not equal (example if there are total
+ * 10 blocks and 3 workers, then as per below calculation each
+ * worker will scan 3 blocks), last worker will be responsible for
+ * scanning remaining blocks. We always consider master backend
+ * as last worker because it will first try to get the tuples
+ * scanned by other workers. For calculation of number of blocks
+ * per worker, an additional worker needs to be consider for
+ * master backend.
+ */
+ pathnode->num_blocks_per_worker = rel->pages / (nWorkers + 1);
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..c8afe99
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,306 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitiateWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_RELID 0
+#define PARALLEL_KEY_TARGETLIST 1
+#define PARALLEL_KEY_QUAL 2
+#define PARALLEL_KEY_RANGETBL 3
+#define PARALLEL_KEY_PARAMS 4
+#define PARALLEL_KEY_BLOCKS 5
+#define PARALLEL_KEY_INST_OPTIONS 6
+#define PARALLEL_KEY_INST_INFO 7
+#define PARALLEL_KEY_TUPLE_QUEUE 8
+#define PARALLEL_SEQSCAN_KEYS 9
+
+static void exec_worker_message(dsm_segment *seg, shm_toc *toc);
+
+/*
+ * InitiateWorkers
+ * It sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitiateWorkers(Index scanrelId, List *targetList, List *qual,
+ List *rangeTable, ParamListInfo params, int instOptions,
+ char **inst_options_space, shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp, BlockNumber numBlocksPerWorker,
+ int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ int i;
+ Size targetlist_len, qual_len, rangetbl_len, params_len;
+ BlockNumber *num_blocks_per_worker;
+ Oid *scanreliddata;
+ char *targetlistdata;
+ char *targetlist_str;
+ char *qualdata;
+ char *qual_str;
+ char *rangetbldata;
+ char *rangetbl_str;
+ char *paramsdata;
+ int *inst_options;
+ char *tuple_queue_space;
+ ParallelContext *pcxt;
+ shm_mq *mq;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(exec_worker_message, nWorkers);
+
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(scanrelId));
+
+ targetlist_str = nodeToString(targetList);
+ targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, targetlist_len);
+
+ qual_str = nodeToString(qual);
+ qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, qual_len);
+
+ rangetbl_str = nodeToString(rangeTable);
+ rangetbl_len = strlen(rangetbl_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, rangetbl_len);
+
+ params_len = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, params_len);
+
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(BlockNumber));
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * nWorkers);
+
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * nWorkers);
+
+ /* keys for parallel sequence scan specific data. */
+ shm_toc_estimate_keys(&pcxt->estimator, PARALLEL_SEQSCAN_KEYS);
+
+ InitializeParallelDSM(pcxt);
+
+ /* Store scan relation id in dynamic shared memory. */
+ scanreliddata = shm_toc_allocate(pcxt->toc, sizeof(Index));
+ *scanreliddata = scanrelId;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_RELID, scanreliddata);
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(pcxt->toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(pcxt->toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUAL, qualdata);
+
+ /* Store range table list in dynamic shared memory. */
+ rangetbldata = shm_toc_allocate(pcxt->toc, rangetbl_len);
+ memcpy(rangetbldata, rangetbl_str, rangetbl_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_RANGETBL, rangetbldata);
+
+ /*
+ * Store parametr's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_len);
+ SerializeBoundParams(params, params_len, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /* Store blocks to be scanned by each worker in dynamic shared memory. */
+ num_blocks_per_worker = shm_toc_allocate(pcxt->toc, sizeof(BlockNumber));
+ *num_blocks_per_worker = numBlocksPerWorker;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BLOCKS, num_blocks_per_worker);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(nWorkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(pcxt);
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], pcxt->worker[i].bgwhandle);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+
+/*
+ * exec_worker_message
+ *
+ * Execute the work assigned to a worker by master backend.
+ */
+void
+exec_worker_message(dsm_segment *seg, shm_toc *toc)
+{
+ char *targetlistdata;
+ char *qualdata;
+ char *rangetbldata;
+ char *paramsdata;
+ char *tuple_queue_space;
+ BlockNumber *num_blocks_per_worker;
+ BlockNumber start_block;
+ BlockNumber end_block;
+ int *inst_options;
+ char *inst_options_space;
+ char *instrument = NULL;
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ Index *scanrelId;
+ List *targetList = NIL;
+ List *qual = NIL;
+ List *rangeTableList = NIL;
+ ParamListInfo params = NULL;
+ worker_stmt *workerstmt;
+
+ scanrelId = shm_toc_lookup(toc, PARALLEL_KEY_RELID);
+ targetlistdata = shm_toc_lookup(toc, PARALLEL_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PARALLEL_KEY_QUAL);
+ rangetbldata = shm_toc_lookup(toc, PARALLEL_KEY_RANGETBL);
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ num_blocks_per_worker = shm_toc_lookup(toc, PARALLEL_KEY_BLOCKS);
+ inst_options = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(mq, MyProc);
+ responseq = shm_mq_attach(mq, seg, NULL);
+
+ end_block = (ParallelWorkerNumber + 1) * (*num_blocks_per_worker);
+ start_block = end_block - (*num_blocks_per_worker);
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_tuple_shm_mq(responseq);
+
+ /* Restore targetList, qual and rangeTableList passed by main backend. */
+ targetList = (List *) stringToNode(targetlistdata);
+ qual = (List *) stringToNode(qualdata);
+ rangeTableList = (List *) stringToNode(rangetbldata);
+ params = RestoreBoundParams(paramsdata);
+
+ workerstmt = palloc(sizeof(worker_stmt));
+
+ workerstmt->scanrelId = *scanrelId;
+ workerstmt->targetList = targetList;
+ workerstmt->qual = qual;
+ workerstmt->rangetableList = rangeTableList;
+ workerstmt->params = params;
+ workerstmt->startBlock = start_block;
+ workerstmt->inst_options = *inst_options;
+ workerstmt->instrument = instrument;
+
+ /*
+ * Last worker should scan all the remaining blocks.
+ *
+ * XXX - It is possible that expected number of workers
+ * won't get started, so to handle such cases master
+ * backend should scan remaining blocks.
+ */
+ workerstmt->endBlock = end_block;
+
+ /* Execute the worker command. */
+ exec_worker_stmt(workerstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 47ed84c..994eeba 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..da6e099 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -148,10 +148,19 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestRemoteExecute:
/*
- * We assume the commandTag is plain ASCII and therefore requires
- * no encoding conversion.
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
*/
- pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ if (is_tuple_shm_mq_enabled())
+ mq_putmessage_direct('C', commandTag, strlen(commandTag) + 1);
+ else
+ {
+ /*
+ * We assume the commandTag is plain ASCII and therefore requires
+ * no encoding conversion.
+ */
+ pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ }
break;
case DestNone:
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index bbad0dc..8c6946b 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1003,7 +1004,7 @@ exec_simple_query(const char *query_string)
/*
* Start the portal. No parameters here.
*/
- PortalStart(portal, NULL, 0, InvalidSnapshot);
+ PortalStart(portal, NULL, 0, InvalidSnapshot, 0);
/*
* Select the appropriate output format: text unless we are doing a
@@ -1132,6 +1133,121 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_worker_stmt(worker_stmt *workerstmt)
+{
+ Portal portal;
+ int16 format = 1;
+ DestReceiver *receiver;
+ bool isTopLevel = true;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(workerstmt);
+
+ /*
+ * Create unnamed portal to run the query or queries in. If there
+ * already is one, silently drop it.
+ */
+ portal = CreatePortal("", true, true);
+ /* Don't display the portal in pg_cursors */
+ portal->visible = false;
+
+ /*
+ * We don't have to copy anything into the portal, because everything
+ * we are passing here is in MessageContext, which will outlive the
+ * portal anyway.
+ */
+ PortalDefineQuery(portal,
+ NULL,
+ "",
+ "",
+ list_make1(planned_stmt),
+ NULL);
+
+ /*
+ * Start the portal. No parameters here.
+ */
+ PortalStart(portal,
+ workerstmt->params,
+ 0,
+ InvalidSnapshot,
+ workerstmt->inst_options);
+
+ /* We always use binary format, for efficiency. */
+ PortalSetResultFormat(portal, 1, &format);
+
+ if (workerstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestRemote);
+ SetRemoteDestReceiverParams(receiver, portal);
+ }
+
+ /*
+ * Only once the portal and destreceiver have been established can
+ * we return to the transaction context. All that stuff needs to
+ * survive an internal commit inside PortalRun!
+ */
+ MemoryContextSwitchTo(oldcontext);
+
+ /*
+ * Run the portal to completion, and then drop it (and the receiver).
+ */
+ (void) PortalRun(portal,
+ FETCH_ALL,
+ isTopLevel,
+ receiver,
+ receiver,
+ NULL);
+
+
+ if (!workerstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (workerstmt->inst_options)
+ memcpy(workerstmt->instrument,
+ portal->queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ PortalDrop(portal, false);
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemote);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
@@ -1735,7 +1851,7 @@ exec_bind_message(StringInfo input_message)
/*
* And we're ready to start portal execution.
*/
- PortalStart(portal, params, 0, InvalidSnapshot);
+ PortalStart(portal, params, 0, InvalidSnapshot, 0);
/*
* Apply the result format requests to the portal.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..5c83799 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -452,12 +452,15 @@ FetchStatementTargetList(Node *stmt)
* presently ignored for non-PORTAL_ONE_SELECT portals (it's only intended
* to be used for cursors).
*
+ * The caller can also provide an options for instrumentation to be passed
+ * to CreateQueryDesc. Most callers should simply pass zero.
+ *
* On return, portal is ready to accept PortalRun() calls, and the result
* tupdesc (if any) is known.
*/
void
PortalStart(Portal portal, ParamListInfo params,
- int eflags, Snapshot snapshot)
+ int eflags, Snapshot snapshot, int inst_options)
{
Portal saveActivePortal;
ResourceOwner saveResourceOwner;
@@ -515,7 +518,7 @@ PortalStart(Portal portal, ParamListInfo params,
InvalidSnapshot,
None_Receiver,
params,
- 0);
+ inst_options);
/*
* If it's a scrollable cursor, executor needs to support
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d9bfa25..b8f90b7 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -630,6 +630,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2445,6 +2447,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2632,6 +2644,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..784cfe0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -287,6 +287,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -497,6 +500,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 939d93d..71ef2c2 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -115,6 +115,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key);
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
+extern void heap_setsyncscan(HeapScanDesc scan, bool sync_scan);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
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
@@ -45,6 +45,8 @@ typedef struct ParallelContext
extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExtension(char *library_name,
char *function_name, int nworkers);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..3c56b49 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -105,4 +105,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..df56cfe
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,44 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ HeapTuple tuple;
+ int num_shm_queues;
+ bool *has_row_description;
+ bool *queue_detached;
+ bool all_queues_detached;
+ bool all_heap_fetched;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..e8522fe 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,6 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index ad7589d..067edbe 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -19,6 +19,13 @@
extern void pq_redirect_to_shm_mq(shm_mq *, shm_mq_handle *);
extern void pq_set_parallel_master(pid_t pid, BackendId backend_id);
+extern int
+mq_putmessage_direct(char msgtype, const char *s, size_t len);
+extern void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh);
+extern bool
+is_tuple_shm_mq_enabled(void);
+
extern void pq_parse_errornotice(StringInfo str, ErrorData *edata);
#endif /* PQMQ_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41288ed..844a9eb 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,12 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1212,6 +1215,24 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+ char *inst_options_space;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..b6f1493 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index 5b096c5..eb8c86a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index b1dfa85..f08448f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,9 +20,13 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
+#include "nodes/params.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "nodes/params.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +160,19 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct worker_stmt
+{
+ Index scanrelId;
+ List *targetList;
+ List *qual;
+ List *rangetableList;
+ ParamListInfo params;
+ BlockNumber startBlock;
+ BlockNumber endBlock;
+ int inst_options;
+ char *instrument;
+} worker_stmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 316c9ce..3354398 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,7 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
#include "utils/lockwaitpolicy.h"
@@ -269,6 +270,8 @@ typedef struct Scan
{
Plan plan;
Index scanrelid; /* relid is index into the range table */
+ BlockNumber startblock; /* block to start seq scan */
+ BlockNumber endblock; /* block upto which scan has to be done */
} Scan;
/* ----------------
@@ -278,6 +281,17 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..576add5 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+ BlockNumber num_blocks_per_worker;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..0b6a469 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..32c3e0d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 082f7d7..a4faf32 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,8 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(worker_stmt *workerstmt);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..91ddffe 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(worker_stmt *workerstmt);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..c0b9b42
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,33 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+extern void InitiateWorkers(Index scanrelId, List *targetList,
+ List *qual, List *rangeTable,
+ ParamListInfo params,
+ int instOptions,
+ char **inst_options_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ BlockNumber numBlocksPerWorker,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/pquery.h b/src/include/tcop/pquery.h
index 8073a6e..d14d876 100644
--- a/src/include/tcop/pquery.h
+++ b/src/include/tcop/pquery.h
@@ -28,7 +28,7 @@ extern List *FetchPortalTargetList(Portal portal);
extern List *FetchStatementTargetList(Node *stmt);
extern void PortalStart(Portal portal, ParamListInfo params,
- int eflags, Snapshot snapshot);
+ int eflags, Snapshot snapshot, int inst_options);
extern void PortalSetResultFormat(Portal portal, int nFormats,
int16 *formats);
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 0a350fd..02cf518 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -83,5 +83,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_worker_stmt(worker_stmt *workerstmt);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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]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.; 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.
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.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
Attachments:
parallel-heap-scan.patchapplication/x-patch; name=parallel-heap-scan.patchDownload
commit 096b3d5bdb4df5de095104fd3f58efa97e08a2ff
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Feb 6 21:19:40 2015 -0500
Support parallel heap scans.
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 50bede8..abfe8c2 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -62,6 +62,7 @@
#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
+#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/datum.h"
#include "utils/inval.h"
@@ -79,8 +80,10 @@ bool synchronize_seqscans = true;
static HeapScanDesc heap_beginscan_internal(Relation relation,
Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync,
bool is_bitmapscan, bool temp_snap);
+static BlockNumber heap_parallelscan_nextpage(ParallelHeapScanDesc);
static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
TransactionId xid, CommandId cid, int options);
static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
@@ -221,7 +224,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool is_rescan)
* results for a non-MVCC snapshot, the caller must hold some higher-level
* lock that ensures the interesting tuple(s) won't change.)
*/
- scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
+ if (scan->rs_parallel != NULL)
+ scan->rs_nblocks = scan->rs_parallel->phs_nblocks;
+ else
+ scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
/*
* If the table is large relative to NBuffers, use a bulk-read access
@@ -480,7 +486,18 @@ heapgettup(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineoff = FirstOffsetNumber; /* first offnum */
scan->rs_inited = true;
@@ -503,6 +520,9 @@ heapgettup(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -655,11 +675,19 @@ heapgettup(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -757,7 +785,18 @@ heapgettup_pagemode(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineindex = 0;
scan->rs_inited = true;
@@ -777,6 +816,9 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -918,11 +960,19 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -1303,7 +1353,7 @@ HeapScanDesc
heap_beginscan(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, false, false);
}
@@ -1313,7 +1363,7 @@ heap_beginscan_catalog(Relation relation, int nkeys, ScanKey key)
Oid relid = RelationGetRelid(relation);
Snapshot snapshot = RegisterSnapshot(GetCatalogSnapshot(relid));
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, false, true);
}
@@ -1322,7 +1372,7 @@ heap_beginscan_strat(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, false, false);
}
@@ -1330,13 +1380,14 @@ HeapScanDesc
heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
false, false, true, false);
}
static HeapScanDesc
heap_beginscan_internal(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync,
bool is_bitmapscan, bool temp_snap)
{
@@ -1364,6 +1415,7 @@ heap_beginscan_internal(Relation relation, Snapshot snapshot,
scan->rs_allow_strat = allow_strat;
scan->rs_allow_sync = allow_sync;
scan->rs_temp_snap = temp_snap;
+ scan->rs_parallel = parallel_scan;
/*
* we can use page-at-a-time mode if it's an MVCC-safe snapshot
@@ -1457,6 +1509,79 @@ heap_endscan(HeapScanDesc scan)
}
/* ----------------
+ * heap_parallelscan_estimate - estimate storage for ParallelHeapScanDesc
+ *
+ * Sadly, this doesn't reduce to a constant, because the size required
+ * to serialize the snapshot can vary.
+ * ----------------
+ */
+Size
+heap_parallelscan_estimate(Snapshot snapshot)
+{
+ return add_size(offsetof(ParallelHeapScanDescData, phs_snapshot_data),
+ EstimateSnapshotSpace(snapshot));
+}
+
+/* ----------------
+ * heap_parallelscan_initialize - initialize ParallelHeapScanDesc
+ *
+ * Must allow as many bytes of shared memory as returned by
+ * heap_parallelscan_estimate. Call this just once in the leader
+ * process; then, individual workers attach via heap_beginscan_parallel.
+ * ----------------
+ */
+void
+heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
+ Snapshot snapshot)
+{
+ target->phs_relid = RelationGetRelid(relation);
+ target->phs_nblocks = RelationGetNumberOfBlocks(relation);
+ SpinLockInit(&target->phs_mutex);
+ target->phs_cblock = 0;
+ SerializeSnapshot(snapshot, target->phs_snapshot_data);
+}
+/* ----------------
+ * heap_parallelscan_nextpage - get the next page to scan
+ *
+ * A return value larger than the number of blocks to be scanned
+ * indicates end of scan. Note, however, that other backends could still
+ * be scanning if they grabbed a page to scan and aren't done with it yet.
+ * ----------------
+ */
+static BlockNumber
+heap_parallelscan_nextpage(ParallelHeapScanDesc parallel_scan)
+{
+ BlockNumber page = InvalidBlockNumber;
+
+ /* we treat InvalidBlockNumber specially here to avoid overflow */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock != InvalidBlockNumber)
+ page = parallel_scan->phs_cblock++;
+ SpinLockRelease(¶llel_scan->phs_mutex);
+
+ return page;
+}
+
+/* ----------------
+ * heap_beginscan_parallel - join a parallel scan
+ *
+ * Caller must hold a suitable lock on the correct relation.
+ * ----------------
+ */
+HeapScanDesc
+heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
+{
+ Snapshot snapshot;
+
+ Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
+ snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
+ RegisterSnapshot(snapshot);
+
+ return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
+ true, true, false, true);
+}
+
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 939d93d..fb2b5f0 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -95,8 +95,9 @@ extern Relation heap_openrv_extended(const RangeVar *relation,
#define heap_close(r,l) relation_close(r,l)
-/* struct definition appears in relscan.h */
+/* struct definitions appear in relscan.h */
typedef struct HeapScanDescData *HeapScanDesc;
+typedef struct ParallelHeapScanDescData *ParallelHeapScanDesc;
/*
* HeapScanIsValid
@@ -119,6 +120,11 @@ extern void heap_rescan(HeapScanDesc scan, ScanKey key);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
+extern Size heap_parallelscan_estimate(Snapshot snapshot);
+extern void heap_parallelscan_initialize(ParallelHeapScanDesc target,
+ Relation relation, Snapshot snapshot);
+extern HeapScanDesc heap_beginscan_parallel(Relation, ParallelHeapScanDesc);
+
extern bool heap_fetch(Relation relation, Snapshot snapshot,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
Relation stats_relation);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..f459020 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,6 +20,15 @@
#include "access/itup.h"
#include "access/tupdesc.h"
+/* Struct for parallel scan setup */
+typedef struct ParallelHeapScanDescData
+{
+ Oid phs_relid;
+ BlockNumber phs_nblocks;
+ slock_t phs_mutex;
+ BlockNumber phs_cblock;
+ char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelHeapScanDescData;
typedef struct HeapScanDescData
{
@@ -48,6 +57,7 @@ typedef struct HeapScanDescData
BlockNumber rs_cblock; /* current block # in scan, if any */
Buffer rs_cbuf; /* current buffer in scan, if any */
/* NB: if rs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
+ ParallelHeapScanDesc rs_parallel; /* parallel scan information */
/* these fields only used in page-at-a-time mode and for bitmap scans */
int rs_cindex; /* current tuple's index in vistuples */
parallel-count.patchapplication/x-patch; name=parallel-count.patchDownload
commit 8d6ad4e1551252e17b7d7609f42f7a24921a2a31
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 08:39:22 2015 -0500
contrib/parallel_count, now using heap_parallel_beginscan
diff --git a/contrib/parallel_count/Makefile b/contrib/parallel_count/Makefile
new file mode 100644
index 0000000..221c569
--- /dev/null
+++ b/contrib/parallel_count/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_count
+OBJS = parallel_count.o $(WIN32RES)
+PGFILEDESC = "parallel_count - simple parallel tuple counter"
+
+EXTENSION = parallel_count
+DATA = parallel_count--1.0.sql
+
+REGRESS = parallel_count
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_count
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_count/parallel_count--1.0.sql b/contrib/parallel_count/parallel_count--1.0.sql
new file mode 100644
index 0000000..a8a6266
--- /dev/null
+++ b/contrib/parallel_count/parallel_count--1.0.sql
@@ -0,0 +1,7 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_count" to load this file. \quit
+
+CREATE FUNCTION parallel_count(rel pg_catalog.regclass,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.int8 STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
diff --git a/contrib/parallel_count/parallel_count.c b/contrib/parallel_count/parallel_count.c
new file mode 100644
index 0000000..06a5ec3
--- /dev/null
+++ b/contrib/parallel_count/parallel_count.c
@@ -0,0 +1,154 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_count.c
+ * simple parallel tuple counter
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_count/parallel_count.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/relscan.h"
+#include "access/xact.h"
+#include "fmgr.h"
+#include "miscadmin.h"
+#include "storage/bufmgr.h"
+#include "storage/spin.h"
+#include "utils/builtins.h"
+#include "utils/guc.h"
+#include "utils/snapmgr.h"
+#include "utils/tqual.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define KEY_SCAN 1
+#define KEY_RESULT 2
+
+void _PG_init(void);
+void count_worker_main(dsm_segment *seg, shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelHeapScanDesc scan,
+ int64 *result);
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ int32 i;
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ Snapshot snapshot;
+ Size pscan_size;
+ ParallelHeapScanDesc pscan;
+ Relation rel;
+ int64 *result = NULL;
+ int64 total = 0;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ rel = relation_open(relid, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_count",
+ "count_worker_main",
+ nworkers);
+
+ snapshot = GetActiveSnapshot();
+ pscan_size = heap_parallelscan_estimate(snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, pscan_size);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ if (nworkers > 0)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator, nworkers * sizeof(int64));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ InitializeParallelDSM(pcxt);
+
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, snapshot);
+ shm_toc_insert(pcxt->toc, KEY_SCAN, pscan);
+ if (nworkers > 0)
+ {
+ result = shm_toc_allocate(pcxt->toc, nworkers * sizeof(int64));
+ shm_toc_insert(pcxt->toc, KEY_RESULT, result);
+ }
+
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, pscan, &total);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ for (i = 0; i < nworkers; ++i)
+ total += result[i];
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(total);
+}
+
+void
+count_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelHeapScanDesc pscan;
+ int64 *result;
+ Relation rel;
+
+ pscan = shm_toc_lookup(toc, KEY_SCAN);
+ Assert(pscan != NULL);
+
+ result = shm_toc_lookup(toc, KEY_RESULT);
+ Assert(result != NULL);
+
+ rel = relation_open(pscan->phs_relid, AccessShareLock);
+ count_helper(rel, pscan, &result[ParallelWorkerNumber]);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelHeapScanDesc pscan, int64 *result)
+{
+ int64 ntuples = 0;
+ HeapScanDesc scan;
+
+ scan = heap_beginscan_parallel(rel, pscan);
+
+ for (;;)
+ {
+ HeapTuple tuple;
+
+ CHECK_FOR_INTERRUPTS();
+
+ tuple = heap_getnext(scan, ForwardScanDirection);
+ if (!HeapTupleIsValid(tuple))
+ break;
+
+ ++ntuples;
+ }
+
+ heap_endscan(scan);
+
+ *result = ntuples;
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, ntuples);
+}
diff --git a/contrib/parallel_count/parallel_count.control b/contrib/parallel_count/parallel_count.control
new file mode 100644
index 0000000..76f332d
--- /dev/null
+++ b/contrib/parallel_count/parallel_count.control
@@ -0,0 +1,4 @@
+comment = 'simple parallel tuple counter'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_count'
+relocatable = true
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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]Actually, there's not really any "maybe" about this. 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.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Feb 7, 2015 at 12:43 AM, 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.
The range table is used by executor for processing qualification, one of
the examples is ExecEvalWholeRowVar(), I don't think we can process
without range table. Apart from above mentioned things we need to pass
Instrumentation structure where each worker needs to update the same,
this is required for Explain statement.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Sun, Feb 8, 2015 at 3:46 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
I think doing it for parallel seq scan as well makes the processing for
worker much more easier like processing for prepared queries
(bind parameters), processing of Explain statement, Qualification,
Projection, decision for processing of junk entries.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Feb 7, 2015 at 2:30 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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:
I think even to make quals work, we need to do few extra things
like setup paramlist, rangetable. Also for quals, we need to fix
function id's by calling fix_opfuncids() and do the stuff what
ExecInit*() function does for quals. I think these extra things
will be required in processing of qualification for seq scan.
Then we need to construct projection info from target list (basically
do the stuff what ExecInit*() function does). After constructing
projectioninfo, we can call ExecProject().
Here we need to take care that functions to collect instrumentation
information like InstrStartNode(), InstrStopNode(), InstrCountFiltered1(),
etc. be called at appropriate places, so that we can collect the same for
Explain statement when requested by master backend.
Then finally after sending tuples need to destroy all the execution
state constructed for fetching tuples.
So to make this work, basically we need to do all important work
that executor does in three different phases initialization of
node, execution of node, ending the node. Ideally, we can make this
work by having code specific to just execution of sequiatial scan,
however it seems to me we again need more such kinds of code
(extracted from core part of executor) to make parallel execution of
other functionalaties like aggregation, partition seq scan, etc.
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).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Feb 6, 2015 at 11:04 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
Master backend shares Targetlist, Qual, Scanrelid, Rangetable, Bind Params,
Info about Scan range (Blocks), Tuple queues, Instrumentation Info
to worker, going by your suggestion, I think we can separate them as below:
1. parallel query - Target list, Qual, Bind Params, Instrumentation Info
2. parallel scan and nodes that returns tuples - scanrelid, range table,
Tuple Queues
3. parallel sequiantial scan specific - Info about Scan range (Blocks)
This is as per current list of things which master backend shares with
worker,
if more things are required, then we can decide in which category it falls
and
add it accordingly.
Is this similar to what you have in mind?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Sun, Feb 8, 2015 at 11:03 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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?
Basically I want to address your concern of setting scan limit via
sequence scan node, one of the ways could be that pass a callback_function
and callback_state to heap_beginscan which will remember that information
in HeapScanDesc and then use in heap_getnext(), now callback_state will
have info about next page which will be updated by callback_function.
We can remember callback_function and callback_state information in
estate which will be set only by parallel worker which means it won't effect
non-parallel case. I think this will be helpful in future as well where we
want
particular scan or sort to use that information to behave as parallel scan
or
sort.
With Regards,
Amit Kapila.
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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 likeSomeUpperLayerNode
|
|
AggCombinerNode
/ \
/ \
/ \
PartialHashAggNode PartialHashAggNode .... .PartialHashAggNode ...
| |
| |
| |
| |
PartialSeqScan PartialSeqScanThe 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 likeSomeUpperLayerNode
|
|
AggCombinerNode
/ \
/ \
/ \
PartialHashAggNode PartialHashAggNode .... .PartialHashAggNode ...
| |
| |
| |
| |
PartialSeqScan PartialSeqScanThe 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Feb 12, 2015 at 2:19 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
It seems to me that we need to use both ways (make heap or other lower
layers aware of parallelism and another one is handle at executor level and
use callback_function and callback_state to make it work) for doing
parallelism. TBH, I think for the matter of this patch we can go either way
and then think more on it as we move ahead to parallelize other operations.
So what I can do is to try using Robert's patch to make heap aware of
parallelism and then see how it comes up?
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.
As per my understanding of the discussion related to this point, I think
there are 3 somewhat related ways to achieve this.
1. Both master and worker runs the same node (ParallelSeqScan) where
the work done by worker (scan chunks of the heap) for this node is
subset of what is done by master (coordinate the data returned by workers +
scan chunks of heap). It seems to me Robert is advocating this approach.
2. Master and worker uses different nodes to operate. Master runs
parallelism
drivingnode (ParallelSeqscan - coordinate the data returned by workers +
scan chunks of heap ) and worker runs some form of Parallelismdriver
node (PartialSeqScan - scan chunks of the heap). It seems to me
Andres is proposing this approach.
3. Same as 2, but modify existing SeqScan node to behave as
PartialSeqScan. This is what I have done in patch.
Correct me or add here if I have misunderstood any thing.
I think going forward (for cases like aggregation) the work done in
Master and Worker node will have substantial differences that it
is better to do the work as part of different nodes in master and
worker.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Feb 9, 2015 at 7:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
rather than Portal-level interfaces. To achieve that I need to add
a new Dest (DestRemoteBackend). For now, I have modified
printtup.c to handle this new destination type similar to what
it does for DestRemote and DestRemoteExecute.
Apart from above, the other major changes to address your concerns
and review comments are:
a. Made InitiateWorkers() and ParallelQueryMain(an entry function for
parallel query execution) modular
b. Adapted the parallel-heap-scan patch posted by Robert upthread
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
c. Now master and worker backend, both run as part of same node
ParallelSeqScan (I have yet to update copy and out funcs for new
parameters), check if you think that is the right way to go. I still
feel it would have been better if master and backend worker runs
as part of different nodes, however this also looks okay for the
purpose of parallel sequential scan.
I have yet to modify the code to allow expressions in projection
and allowing joins, I think these are related to allow-parallel-safety
patch, I will once take a look at that patch and then modify
accordingly.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v7.patchapplication/octet-stream; name=parallel_seqscan_v7.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..a032afb 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -82,7 +82,7 @@ printtup_create_DR(CommandDest dest)
/*
* Send T message automatically if DestRemote, but not if
- * DestRemoteExecute
+ * DestRemoteExecute or DestRemoteBackend
*/
self->sendDescrip = (dest == DestRemote);
@@ -95,7 +95,8 @@ printtup_create_DR(CommandDest dest)
}
/*
- * Set parameters for a DestRemote (or DestRemoteExecute) receiver
+ * Set parameters for a DestRemote (or DestRemoteExecute or DestRemoteBackend)
+ * receiver
*/
void
SetRemoteDestReceiverParams(DestReceiver *self, Portal portal)
@@ -103,7 +104,8 @@ SetRemoteDestReceiverParams(DestReceiver *self, Portal portal)
DR_printtup *myState = (DR_printtup *) self;
Assert(myState->pub.mydest == DestRemote ||
- myState->pub.mydest == DestRemoteExecute);
+ myState->pub.mydest == DestRemoteExecute ||
+ myState->pub.mydest == DestRemoteBackend);
myState->portal = portal;
@@ -243,7 +245,19 @@ SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist, int16 *formats)
pq_sendint(&buf, 0, 2);
}
}
- pq_endmessage(&buf);
+
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
}
/*
@@ -252,9 +266,15 @@ SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist, int16 *formats)
static void
printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
{
- int16 *formats = myState->portal->formats;
+ int16 *formats;
int i;
+ /* Remote backend always uses binary format to communicate. */
+ if (myState->pub.mydest == DestRemoteBackend)
+ formats = NULL;
+ else
+ formats = myState->portal->formats;
+
/* get rid of any old data */
if (myState->myinfo)
pfree(myState->myinfo);
@@ -271,7 +291,12 @@ printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
for (i = 0; i < numAttrs; i++)
{
PrinttupAttrInfo *thisState = myState->myinfo + i;
- int16 format = (formats ? formats[i] : 0);
+ int16 format;
+
+ if (myState->pub.mydest == DestRemoteBackend)
+ format = (formats ? formats[i] : 1);
+ else
+ format = (formats ? formats[i] : 0);
thisState->format = format;
if (format == 0)
@@ -371,7 +396,18 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
}
}
- pq_endmessage(&buf);
+ /*
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
+ */
+ if (is_tuple_shm_mq_enabled())
+ {
+ mq_putmessage_direct(buf.cursor, buf.data, buf.len);
+ pfree(buf.data);
+ buf.data = NULL;
+ }
+ else
+ pq_endmessage(&buf);
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..116a717
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,339 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId);
+
+/*
+ * shm_beginscan
+ *
+ * Initializes the shared memory scan descriptor to retrieve tuples
+ * from worker backends.
+ */
+ShmScanDesc
+shm_beginscan(int num_queues)
+{
+ ShmScanDesc shmscan;
+
+ shmscan = palloc(sizeof(ShmScanDescData));
+
+ shmscan->num_shm_queues = num_queues;
+ shmscan->ss_cqueue = -1;
+ shmscan->shmscan_inited = false;
+
+ return shmscan;
+}
+
+/*
+ * ExecInitWorkerResult
+ *
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers)
+{
+ worker_result workerResult;
+ int i;
+ int natts = tupdesc->natts;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+ workerResult->receive_functions = palloc(sizeof(FmgrInfo) * natts);
+ workerResult->typioparams = palloc(sizeof(Oid) * natts);
+ workerResult->num_shm_queues = nWorkers;
+ workerResult->queue_detached = palloc0(sizeof(bool) * nWorkers);
+
+ for (i = 0; i < natts; ++i)
+ {
+ Oid receive_function_id;
+
+ getTypeBinaryInputInfo(tupdesc->attrs[i]->atttypid,
+ &receive_function_id,
+ &workerResult->typioparams[i]);
+ fmgr_info(receive_function_id, &workerResult->receive_functions[i]);
+ }
+
+ return workerResult;
+}
+
+
+/*
+ * shm_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap)
+{
+ shm_mq_result res;
+ Size nbytes;
+ void *data;
+ StringInfoData msg;
+ int queueId = 0;
+
+ /*
+ * calculate next starting queue used for fetching tuples
+ */
+ if(!shmScan->shmscan_inited)
+ {
+ shmScan->shmscan_inited = true;
+ Assert(shmScan->num_shm_queues > 0);
+ queueId = 0;
+ }
+ else
+ queueId = shmScan->ss_cqueue;
+
+ /* Read and processes messages from the shared memory queues. */
+ for(;;)
+ {
+ if (!resultState->all_queues_detached)
+ {
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+
+ /*
+ * Don't fetch from detached queue. This loop could continue
+ * forever, if we reach a situation such that all queue's are
+ * detached, however we won't reach here if that is the case.
+ */
+ while (resultState->queue_detached[queueId])
+ {
+ ++queueId;
+ if (queueId == shmScan->num_shm_queues)
+ queueId = 0;
+ }
+
+ for (;;)
+ {
+ /*
+ * mark current queue used for fetching tuples, this is used
+ * to fetch consecutive tuples from queue used in previous
+ * fetch.
+ */
+ shmScan->ss_cqueue = queueId;
+
+ /* Get next message. */
+ res = shm_mq_receive(responseq[queueId], &nbytes, &data, true);
+ if (res == SHM_MQ_DETACHED)
+ {
+ /*
+ * mark the queue that got detached, so that we don't
+ * try to fetch from it again.
+ */
+ resultState->queue_detached[queueId] = true;
+ --resultState->num_shm_queues;
+ /*
+ * if we have exhausted data from all worker queues, then don't
+ * process data from queues.
+ */
+ if (resultState->num_shm_queues <= 0)
+ resultState->all_queues_detached = true;
+ break;
+ }
+ else if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ bool rettuple;
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ rettuple = HandleParallelTupleMessage(resultState, tupdesc, &msg, queueId);
+ pfree(msg.data);
+ if (rettuple)
+ {
+ *fromheap = false;
+ return resultState->tuple;
+ }
+ }
+ }
+ }
+
+ /*
+ * if we have checked all the message queue's and didn't find
+ * any message or we have already fetched all the data from queue's,
+ * then it's time to fetch directly from heap. Reset the current
+ * queue as the first queue from which we need to receive tuples.
+ */
+ if ((queueId == shmScan->num_shm_queues - 1 ||
+ resultState->all_queues_detached) &&
+ !resultState->all_heap_fetched)
+ {
+ HeapTuple tuple;
+ shmScan->ss_cqueue = 0;
+ tuple = heap_getnext(scanDesc, direction);
+ if (tuple)
+ {
+ *fromheap = true;
+ return tuple;
+ }
+ else if (tuple == NULL && resultState->all_queues_detached)
+ break;
+ else
+ resultState->all_heap_fetched = true;
+ }
+ else if (resultState->all_queues_detached &&
+ resultState->all_heap_fetched)
+ break;
+
+ /* check the data in next queue. */
+ ++queueId;
+ }
+
+ return NULL;
+}
+
+/*
+ * HandleParallelTupleMessage
+ *
+ * Handle a single tuple related protocol message received from
+ * a single parallel worker.
+ */
+static bool
+HandleParallelTupleMessage(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ char msgtype;
+ bool rettuple = false;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ /* Dispatch on message type. */
+ switch (msgtype)
+ {
+ case 'D':
+ {
+ /* Handle DataRow message. */
+ resultState->tuple = form_result_tuple(resultState, tupdesc, msg, queueId);
+ rettuple = true;
+ break;
+ }
+ case 'C':
+ {
+ /*
+ * Handle CommandComplete message. Ignore tags sent by
+ * worker backend as we are anyway going to use tag of
+ * master backend for sending the same to client.
+ */
+ (void) pq_getmsgstring(msg);
+ break;
+ }
+ case 'G':
+ case 'H':
+ case 'W':
+ {
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ errmsg("COPY protocol not allowed in worker")));
+ }
+ default:
+ elog(WARNING, "unknown message type: %c", msg->data[0]);
+ break;
+ }
+
+ return rettuple;
+}
+
+/*
+ * form_result_tuple
+ *
+ * Parse a DataRow message and form a result tuple.
+ */
+static HeapTuple
+form_result_tuple(worker_result resultState, TupleDesc tupdesc,
+ StringInfo msg, int queueId)
+{
+ /* Handle DataRow message. */
+ int16 natts = pq_getmsgint(msg, 2);
+ int16 i;
+ Datum *values = NULL;
+ bool *isnull = NULL;
+ HeapTuple tuple;
+ StringInfoData buf;
+
+ if (natts != tupdesc->natts)
+ elog(ERROR, "malformed DataRow");
+ if (natts > 0)
+ {
+ values = palloc(natts * sizeof(Datum));
+ isnull = palloc(natts * sizeof(bool));
+ }
+ initStringInfo(&buf);
+
+ for (i = 0; i < natts; ++i)
+ {
+ int32 bytes = pq_getmsgint(msg, 4);
+
+ if (bytes < 0)
+ {
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ NULL,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = true;
+ }
+ else
+ {
+ resetStringInfo(&buf);
+ appendBinaryStringInfo(&buf, pq_getmsgbytes(msg, bytes), bytes);
+ values[i] = ReceiveFunctionCall(&resultState->receive_functions[i],
+ &buf,
+ resultState->typioparams[i],
+ tupdesc->attrs[i]->atttypmod);
+ isnull[i] = false;
+ }
+ }
+
+ pq_getmsgend(msg);
+
+ tuple = heap_form_tuple(tupdesc, values, isnull);
+
+ /*
+ * Release locally palloc'd space. XXX would probably be good to pfree
+ * values of pass-by-reference datums, as well.
+ */
+ pfree(values);
+ pfree(isnull);
+
+ pfree(buf.data);
+
+ return tuple;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 7cfc9bb..8b85e97 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -917,6 +918,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_ParallelSeqScan:
+ pname = sname = "Parallel Seq Scan";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1066,6 +1070,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1207,6 +1212,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_ParallelSeqScan)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((ParallelSeqScanState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((ParallelSeqScanState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1332,6 +1355,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_ParallelSeqScan:
+ 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",
+ ((ParallelSeqScan *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2224,6 +2255,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..9a8ca75 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -21,7 +21,7 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeSeqscan.o nodeParallelSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..f77a77f 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeParallelSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +191,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_ParallelSeqScan:
+ result = (PlanState *) ExecInitParallelSeqScan((ParallelSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +412,10 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ result = ExecParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +654,10 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_ParallelSeqScanState:
+ ExecEndParallelSeqScan((ParallelSeqScanState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..229302d 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,8 +191,17 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification as for them the same is done by worker
+ * backend. This case will happen only for parallel query where we push
+ * down the qualification.
+ * XXX - We can do this optimization for projection as well, but for
+ * now it is okay, as we don't allow parallel query if there are
+ * expressions involved in target list.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..b7898a5 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -21,6 +21,8 @@ BufferUsage pgBufferUsage;
static void BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add, const BufferUsage *sub);
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
/* Allocate new instrumentation structure(s) */
@@ -127,6 +129,28 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
static void
BufferUsageAccumDiff(BufferUsage *dst,
@@ -148,3 +172,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
\ No newline at end of file
diff --git a/src/backend/executor/nodeParallelSeqscan.c b/src/backend/executor/nodeParallelSeqscan.c
new file mode 100644
index 0000000..397a47d
--- /dev/null
+++ b/src/backend/executor/nodeParallelSeqscan.c
@@ -0,0 +1,364 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeParallelSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeParallelSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecParallelSeqScan scans a relation.
+ * ParallelSeqNext retrieve next tuple from either heap or shared memory segment.
+ * ExecInitParallelSeqScan creates and initializes a parallel seqscan node.
+ * ExecEndParallelSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ParallelSeqNext
+ *
+ * This is a workhorse for ExecParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+ParallelSeqNext(ParallelSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ if(((ParallelSeqScan*)node->ss.ps.plan)->shm_toc_key)
+ {
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+ }
+ else
+ {
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc, node->pss_currentShmScanDesc,
+ node->pss_workerResult,
+ node->responseq,
+ node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ direction, &fromheap);
+ }
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ fromheap ? scandesc->rs_cbuf : InvalidBuffer, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * ParallelSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+ParallelSeqRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, ParallelSeqScan never use keys in
+ * shm_beginscan/heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitParallelScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitParallelScanRelation(ParallelSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * For Explain statement, we don't want to initialize workers as
+ * those are maily needed to execute the plan, however scan descriptor
+ * still needs to be initialized for the purpose of InitNode functionality
+ * (as EnNode functionality assumes that scan descriptor and scan relation
+ * must be initialized, probably we can change that but that will make
+ * the code EndParallelSeqScan look different than other node's end
+ * functionality.
+ *
+ * XXX - If we want executorstart to initilize workers as well, then we
+ * need to have a provision for waiting till all the workers get started
+ * otherwise while doing endscan, it will try to wait for termination of
+ * workers which are not even started (and will neither get started).
+ */
+ if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
+ {
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+ }
+ else
+ {
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory.
+ */
+ if (((ParallelSeqScan *) node->ss.ps.plan)->shm_toc_key != 0)
+ {
+ Assert(!pscan);
+
+ pscan = shm_toc_lookup(((ParallelSeqScan *) node->ss.ps.plan)->toc,
+ ((ParallelSeqScan *) node->ss.ps.plan)->shm_toc_key);
+ }
+ else
+ {
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(((SeqScan *) node->ss.ps.plan)->scanrelid,
+ node->ss.ps.plan->targetlist,
+ node->ss.ps.plan->qual,
+ estate,
+ currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ &pscan,
+ ((ParallelSeqScan *)(node->ss.ps.plan))->num_workers);
+ }
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+ }
+
+ node->ss.ss_currentRelation = currentRelation;
+ node->ss.ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * InitShmScan
+ *
+ * Set up to access the scan for shared memory segment.
+ * ----------------------------------------------------------------
+ */
+static void
+InitShmScan(ParallelSeqScanState *node)
+{
+ ShmScanDesc currentShmScanDesc;
+ worker_result workerResult;
+
+ /*
+ * Shared memory scan needs to be initialized only for
+ * master backend as worker backend scans only heap.
+ */
+ if (((ParallelSeqScan *) node->ss.ps.plan)->shm_toc_key == 0)
+ {
+ /*
+ * Use result tuple descriptor to fetch data from shared memory queues
+ * as the worker backend's would have put the data after projection.
+ * Number of queues must be equal to number of worker backend's.
+ */
+ currentShmScanDesc = shm_beginscan(node->pcxt->nworkers);
+ workerResult = ExecInitWorkerResult(node->ss.ps.ps_ResultTupleSlot->tts_tupleDescriptor,
+ node->pcxt->nworkers);
+
+ node->pss_currentShmScanDesc = currentShmScanDesc;
+ node->pss_workerResult = workerResult;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitParallelSeqScan
+ * ----------------------------------------------------------------
+ */
+ParallelSeqScanState *
+ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags)
+{
+ ParallelSeqScanState *parallelscanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ parallelscanstate = makeNode(ParallelSeqScanState);
+ parallelscanstate->ss.ps.plan = (Plan *) node;
+ parallelscanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, ¶llelscanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ parallelscanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) parallelscanstate);
+ parallelscanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) parallelscanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, ¶llelscanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, ¶llelscanstate->ss);
+
+ InitParallelScanRelation(parallelscanstate, estate, eflags);
+
+ parallelscanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(¶llelscanstate->ss.ps);
+ ExecAssignScanProjectionInfo(¶llelscanstate->ss);
+
+ /*
+ * For Explain, we don't initialize the parallel workers, so
+ * accordingly don't need to initialize the shared memory scan.
+ */
+ if (!(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ InitShmScan(parallelscanstate);
+
+ return parallelscanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecParallelSeqScan(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecParallelSeqScan(ParallelSeqScanState *node)
+{
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) ParallelSeqNext,
+ (ExecScanRecheckMtd) ParallelSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndParallelSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndParallelSeqScan(ParallelSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt)
+ {
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index f12f2d5..cfab8b5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -26,6 +26,8 @@ static bool pq_mq_busy = false;
static pid_t pq_mq_parallel_master_pid = 0;
static pid_t pq_mq_parallel_master_backend_id = InvalidBackendId;
+static shm_mq_handle *pq_mq_tuple_handle = NULL;
+
static void mq_comm_reset(void);
static int mq_flush(void);
static int mq_flush_if_writable(void);
@@ -61,6 +63,26 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
}
/*
+ * Arrange to send some frontend/backend protocol messages to a shared-memory
+ * tuple message queue.
+ */
+void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh)
+{
+ pq_mq_tuple_handle = mqh;
+}
+
+/*
+ * Check if tuples can be sent through tuple shared-memory
+ * message queue.
+ */
+bool
+is_tuple_shm_mq_enabled(void)
+{
+ return pq_mq_tuple_handle ? true : false;
+}
+
+/*
* Arrange to SendProcSignal() to the parallel master each time we transmit
* message data via the shm_mq.
*/
@@ -161,6 +183,42 @@ mq_putmessage(char msgtype, const char *s, size_t len)
return 0;
}
+/*
+ * Transmit a libpq protocol message to the shared memory message queue
+ * via pq_mq_tuple_handle. We don't include a length word, because the
+ * receiver will know the length of the message from shm_mq_receive().
+ */
+int
+mq_putmessage_direct(char msgtype, const char *s, size_t len)
+{
+ shm_mq_iovec iov[2];
+ shm_mq_result result;
+
+ iov[0].data = &msgtype;
+ iov[0].len = 1;
+ iov[1].data = s;
+ iov[1].len = len;
+
+ Assert(pq_mq_tuple_handle != NULL);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_tuple_handle, iov, 2, true);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ Assert(result == SHM_MQ_SUCCESS || result == SHM_MQ_DETACHED);
+ if (result != SHM_MQ_SUCCESS)
+ return EOF;
+ return 0;
+}
+
static void
mq_putmessage_noblock(char msgtype, const char *s, size_t len)
{
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index f1a24f5..5846f22 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -352,6 +352,27 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyParallelSeqScan
+ */
+static ParallelSeqScan *
+_copyParallelSeqScan(const ParallelSeqScan *from)
+{
+ ParallelSeqScan *newnode = makeNode(ParallelSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4039,6 +4060,9 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_ParallelSeqScan:
+ retval = _copyParallelSeqScan(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index dd1278b..35c2e1e 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -437,6 +437,16 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outParallelSeqScan(StringInfo str, const ParallelSeqScan *node)
+{
+ WRITE_NODE_TYPE("PARALLELSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2851,6 +2861,9 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_ParallelSeqScan:
+ _outParallelSeqScan(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index 2f2f5ed..7ecaa7f 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -74,3 +87,187 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ /* sizeof(ParamListInfoData) includes the first array element */
+ size = sizeof(ParamListInfoData) +
+ (num_params - 1) * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 020558b..4abfd25 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +227,73 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_parallelseqscan
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+ double spc_seq_page_cost;
+ QualCost qpqual_cost;
+ Cost cpu_per_tuple;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ if (!enable_seqscan)
+ startup_cost += disable_cost;
+
+ /* fetch estimated page cost for tablespace containing table */
+ get_tablespace_page_costs(baserel->reltablespace,
+ NULL,
+ &spc_seq_page_cost);
+
+ /*
+ * disk costs
+ */
+ run_cost += spc_seq_page_cost * baserel->pages;
+
+ /* CPU costs */
+ get_restriction_qual_cost(root, baserel, param_info, &qpqual_cost);
+
+ startup_cost += qpqual_cost.startup;
+ cpu_per_tuple = cpu_tuple_cost + qpqual_cost.per_tuple;
+ run_cost += cpu_per_tuple * baserel->tuples;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..fda6f40
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,148 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * IsTargetListContainNonVars -
+ * Check if target list contain non-var entries.
+ */
+static bool
+IsTargetListContainNonVars(List *targetlist)
+{
+ ListCell *l;
+
+ foreach(l, targetlist)
+ {
+ TargetEntry *te = (TargetEntry *) lfirst(l);
+
+ if (!IsA(te, TargetEntry))
+ continue; /* probably should never happen */
+ if (!IsA(te->expr, Var))
+ return true;
+ }
+ return false;
+}
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * parallel scan is not supported for non-var target list.
+ *
+ * XXX - This is to keep the implementation simple, we can do this
+ * in future. Here we are checking by passing root->parse->targetList
+ * instead of rel->reltargetlist because rel->targetlist always contains
+ * Vars (refer build_base_rel_tlists).
+ */
+ if (IsTargetListContainNonVars(root->parse->targetList))
+ return;
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ add_path(rel, (Path *) create_parallelseqscan_path(root, rel,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 655be81..a8a626e 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,9 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_parallelseqscan_plan(PlannerInfo *root,
+ ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +103,9 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static ParallelSeqScan *make_parallelseqscan(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ shm_toc *toc, uint64 shm_toc_key);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +234,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +350,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_ParallelSeqScan:
+ plan = (Plan *) create_parallelseqscan_plan(root,
+ (ParallelSeqPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +560,7 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_ParallelSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1148,70 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_worker_seqscan_plan
+ *
+ * Returns a parallel seqscan plan for the base relation scanned
+ * by worker with restriction clauses 'qual' and targetlist 'tlist'.
+ */
+Scan *
+create_worker_seqscan_plan(ParallelScanStmt *parallelscan)
+{
+ Scan *scan_plan;
+
+ scan_plan = (Scan*) make_parallelseqscan(parallelscan->targetList,
+ parallelscan->qual,
+ parallelscan->scanrelId,
+ 0,
+ parallelscan->toc,
+ parallelscan->shm_toc_scan_key);
+
+ return scan_plan;
+}
+
+/*
+ * create_parallelseqscan_plan
+ *
+ * Returns a parallel seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_parallelseqscan_plan(PlannerInfo *root, ParallelSeqPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->path.param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_parallelseqscan(tlist,
+ scan_clauses,
+ scan_relid,
+ best_path->num_workers,
+ NULL,
+ 0);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3318,6 +3397,30 @@ make_seqscan(List *qptlist,
return node;
}
+static ParallelSeqScan *
+make_parallelseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ shm_toc *toc,
+ uint64 shm_toc_key)
+{
+ ParallelSeqScan *node = makeNode(ParallelSeqScan);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+ node->toc = toc;
+ node->shm_toc_key = shm_toc_key;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 9cbbcfb..f9be27f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,74 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+/*
+ * create_worker_seqscan_plannedstmt
+ *
+ * Returns a planned statement to be used by worker for execution.
+ * Instead of master backend forming and passing the planned statement
+ * to each worker, it just passes required information and PlannedStmt
+ * is then constructed by worker. The reason for doing so is that
+ * master backend plan doesn't contain the subplans for each worker.
+ * In future, if there is a need for doing so, we might want to
+ * change the implementation master backend will pass the planned
+ * statement directly.
+ */
+PlannedStmt *
+create_worker_seqscan_plannedstmt(ParallelScanStmt *parallelscan)
+{
+ Plan *scan_plan;
+ PlannedStmt *result;
+ ListCell *tlist;
+ Oid reloid;
+
+ /* get the relid to save the same as part of planned statement. */
+ reloid = getrelid(parallelscan->scanrelId, parallelscan->rangetableList);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) parallelscan->qual);
+ fix_opfuncids((Node*) parallelscan->targetList);
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, parallelscan->targetList)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ scan_plan = (Plan*) create_worker_seqscan_plan(parallelscan);
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) scan_plan;
+ result->rtable = parallelscan->rangetableList;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->relationOids = lappend_oid(result->relationOids, reloid);
+ result->invalItems = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to get hasRowSecurity passed from master
+ * backend as this is used only for invalidation and in
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 7703946..3a44aef 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -436,6 +436,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_ParallelSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 78fb6b1..c35f934 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2163,6 +2163,7 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_ParallelSeqScan:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..ea3b865 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,30 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_parallelseqscan_path
+ *
+ * Creates a path corresponding to a parallel sequential scan, returning the
+ * pathnode.
+ */
+ParallelSeqPath *
+create_parallelseqscan_path(PlannerInfo *root, RelOptInfo *rel, int nWorkers)
+{
+ ParallelSeqPath *pathnode = makeNode(ParallelSeqPath);
+
+ pathnode->path.pathtype = T_ParallelSeqScan;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->num_workers = nWorkers;
+
+ cost_parallelseqscan(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..890a0d5
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,562 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeParallelSeqscan.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_SCANRELID 0
+#define PARALLEL_KEY_TARGETLIST 1
+#define PARALLEL_KEY_QUAL 2
+#define PARALLEL_KEY_RANGETBL 3
+#define PARALLEL_KEY_PARAMS 4
+#define PARALLEL_KEY_INST_OPTIONS 5
+#define PARALLEL_KEY_INST_INFO 6
+#define PARALLEL_KEY_TUPLE_QUEUE 7
+#define PARALLEL_KEY_SCAN 8
+#define PARALLEL_KEY_OPERATION 9
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void RestoreAndExecuteParallelScan(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelQueryElemsSpace(ParallelContext *pcxt,
+ char *targetlist_str, char *qual_str,
+ Size *targetlist_len, Size *qual_len);
+static void
+StoreParallelQueryElems(ParallelContext *pcxt,
+ char *targetlist_str, char *qual_str,
+ Size targetlist_len, Size qual_len);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_len);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_len,
+ char **inst_options_space);
+static void
+EstimateParallelSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ Index scanrelId, char *rangetbl_str,
+ Size *rangetbl_len, Size *pscan_size);
+static void
+StoreParallelSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ Index scanrelId, char *rangetbl_str,
+ ParallelHeapScanDesc *pscan,
+ Size rangetbl_len, Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueueAndStartWorkers(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetParallelQueryElems(shm_toc *toc, List **targetList, List **qual);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument);
+static void
+GetParallelSeqScanInfo(shm_toc *toc, Index *scanrelId,
+ List **rangeTableList);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq);
+
+/*
+ * EstimateParallelQueryElemsSpace
+ *
+ * Estimate the amount of space required to record information of
+ * query elements that need to be copied to parallel workers.
+ */
+void
+EstimateParallelQueryElemsSpace(ParallelContext *pcxt,
+ char *targetlist_str, char *qual_str,
+ Size *targetlist_len, Size *qual_len)
+{
+ *targetlist_len = strlen(targetlist_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *targetlist_len);
+
+ *qual_len = strlen(qual_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *qual_len);
+
+ /* keys for parallel query elements. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StoreParallelQueryElems
+ *
+ * Sets up target list and qualification required for parallel
+ * execution.
+ */
+void
+StoreParallelQueryElems(ParallelContext *pcxt,
+ char *targetlist_str, char *qual_str,
+ Size targetlist_len, Size qual_len)
+{
+ char *targetlistdata;
+ char *qualdata;
+
+ /* Store target list in dynamic shared memory. */
+ targetlistdata = shm_toc_allocate(pcxt->toc, targetlist_len);
+ memcpy(targetlistdata, targetlist_str, targetlist_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TARGETLIST, targetlistdata);
+
+ /* Store qual list in dynamic shared memory. */
+ qualdata = shm_toc_allocate(pcxt->toc, qual_len);
+ memcpy(qualdata, qual_str, qual_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_QUAL, qualdata);
+}
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_len)
+{
+ *params_len = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_len);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_len,
+ char **inst_options_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_len);
+ SerializeBoundParams(params, params_len, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimateParallelSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * scanrelId, rangetable and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimateParallelSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ Index scanrelId, char *rangetbl_str,
+ Size *rangetbl_len, Size *pscan_size)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(NodeTag));
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(scanrelId));
+
+ *rangetbl_len = strlen(rangetbl_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *rangetbl_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 4);
+}
+
+/*
+ * StoreParallelSeqScan
+ *
+ * Sets up the scanrelid, rangetable entries and block range
+ * for parallel sequence scan.
+ */
+void
+StoreParallelSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ Index scanrelId, char *rangetbl_str,
+ ParallelHeapScanDesc *pscan,
+ Size rangetbl_len, Size pscan_size)
+{
+ NodeTag *nodetype;
+ Oid *scanreliddata;
+ char *rangetbldata;
+
+ /* Store sequence scan Nodetag in dynamic shared memory. */
+ nodetype = shm_toc_allocate(pcxt->toc, sizeof(NodeTag));
+ *nodetype = T_ParallelSeqScan;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_OPERATION, nodetype);
+
+ /* Store scan relation id in dynamic shared memory. */
+ scanreliddata = shm_toc_allocate(pcxt->toc, sizeof(Index));
+ *scanreliddata = scanrelId;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCANRELID, scanreliddata);
+
+ /* Store range table list in dynamic shared memory. */
+ rangetbldata = shm_toc_allocate(pcxt->toc, rangetbl_len);
+ memcpy(rangetbldata, rangetbl_str, rangetbl_len);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_RANGETBL, rangetbldata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ *pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(*pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, *pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueueAndStartWorkers
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ * This function must be called after setting up all the other
+ * necessary parallel execution related information as it start
+ * the workers after which we can't initialize or pass the parallel
+ * state information.
+ */
+void
+StoreResponseQueueAndStartWorkers(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(pcxt);
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ shm_mq_set_handle((*responseqp)[i], pcxt->worker[i].bgwhandle);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Index scanrelId, List *targetList, List *qual,
+ EState *estate, Relation rel, char **inst_options_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ Size targetlist_len, qual_len, rangetbl_len, params_len, pscan_size;
+ char *targetlist_str;
+ char *qual_str;
+ char *rangetbl_str;
+ ParallelContext *pcxt;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ /* Estimate space for parallel seq. scan specific contents. */
+ targetlist_str = nodeToString(targetList);
+ qual_str = nodeToString(qual);
+ EstimateParallelQueryElemsSpace(pcxt, targetlist_str, qual_str,
+ &targetlist_len, &qual_len);
+
+ rangetbl_str = nodeToString(estate->es_range_table);
+ EstimateParallelSeqScanSpace(pcxt, estate, scanrelId, rangetbl_str,
+ &rangetbl_len, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_len);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StoreParallelQueryElems(pcxt, targetlist_str, qual_str,
+ targetlist_len, qual_len);
+ StoreParallelSeqScan(pcxt, estate, rel, scanrelId, rangetbl_str,
+ pscan, rangetbl_len, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_len, inst_options_space);
+ StoreResponseQueueAndStartWorkers(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelQueryElems
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the targetlist and qualification list required
+ * to perform parallel operation.
+ */
+void
+GetParallelQueryElems(shm_toc *toc, List **targetList, List **qual)
+{
+ char *targetlistdata;
+ char *qualdata;
+
+ targetlistdata = shm_toc_lookup(toc, PARALLEL_KEY_TARGETLIST);
+ qualdata = shm_toc_lookup(toc, PARALLEL_KEY_QUAL);
+
+ *targetList = (List *) stringToNode(targetlistdata);
+ *qual = (List *) stringToNode(qualdata);
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+}
+
+/*
+ * GetParallelSeqScanInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the scanrelId and rangeTable required to perform
+ * parallel sequential scan.
+ */
+void
+GetParallelSeqScanInfo(shm_toc *toc, Index *scanrelId,
+ List **rangeTableList)
+{
+ char *rangetbldata;
+ Index *scanrel;
+
+ scanrel = shm_toc_lookup(toc, PARALLEL_KEY_SCANRELID);
+ rangetbldata = shm_toc_lookup(toc, PARALLEL_KEY_RANGETBL);
+
+ *scanrelId = *scanrel;
+ *rangeTableList = (List *) stringToNode(rangetbldata);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq)
+{
+ char *tuple_queue_space;
+ shm_mq_handle *responseq;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ responseq = shm_mq_attach(*mq, seg, NULL);
+
+ /* Redirect protocol messages to responseq. */
+ pq_redirect_to_tuple_shm_mq(responseq);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ NodeTag *nodetype;
+
+ nodetype = shm_toc_lookup(toc, PARALLEL_KEY_OPERATION);
+
+ switch (*nodetype)
+ {
+ case T_ParallelSeqScan:
+ RestoreAndExecuteParallelScan(seg, toc);
+ break;
+ default:
+ elog(ERROR, "unrecognized node type: %d", (int) *nodetype);
+ break;
+ }
+}
+
+/*
+ * RestoreAndExecuteParallelScan
+ *
+ * Lookup the parallel sequence scan related parameters
+ * from dynamic shared memory segment and setup the
+ * statement to execute the scan.
+ */
+void
+RestoreAndExecuteParallelScan(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ List *targetList = NIL;
+ List *qual = NIL;
+ List *rangeTableList = NIL;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ Index scanrelId;
+ ParallelScanStmt *parallelscan;
+
+ SetupResponseQueue(seg, toc, &mq);
+
+ GetParallelQueryElems(toc, &targetList, &qual);
+ GetParallelSeqScanInfo(toc, &scanrelId, &rangeTableList);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options, &instrument);
+
+ parallelscan = palloc(sizeof(ParallelScanStmt));
+
+ parallelscan->scanrelId = scanrelId;
+ parallelscan->targetList = targetList;
+ parallelscan->qual = qual;
+ parallelscan->rangetableList = rangeTableList;
+ parallelscan->params = params;
+ parallelscan->inst_options = inst_options;
+ parallelscan->instrument = instrument;
+ parallelscan->toc = toc;
+ parallelscan->shm_toc_scan_key = PARALLEL_KEY_SCAN;
+
+ /* Execute the worker command. */
+ exec_parallel_scan(parallelscan);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index ac431e5..4c303dd 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..e7ebc1f 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -104,6 +104,7 @@ CreateDestReceiver(CommandDest dest)
{
case DestRemote:
case DestRemoteExecute:
+ case DestRemoteBackend:
return printtup_create_DR(dest);
case DestNone:
@@ -146,12 +147,22 @@ EndCommand(const char *commandTag, CommandDest dest)
{
case DestRemote:
case DestRemoteExecute:
+ case DestRemoteBackend:
/*
- * We assume the commandTag is plain ASCII and therefore requires
- * no encoding conversion.
+ * Send the message via shared-memory tuple queue, if the same
+ * is enabled.
*/
- pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ if (is_tuple_shm_mq_enabled())
+ mq_putmessage_direct('C', commandTag, strlen(commandTag) + 1);
+ else
+ {
+ /*
+ * We assume the commandTag is plain ASCII and therefore requires
+ * no encoding conversion.
+ */
+ pq_putmessage('C', commandTag, strlen(commandTag) + 1);
+ }
break;
case DestNone:
@@ -204,6 +215,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestRemoteBackend:
break;
}
}
@@ -248,6 +260,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestRemoteBackend:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 8899448..2e42aa2 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -55,6 +55,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1191,6 +1192,90 @@ exec_simple_query(const char *query_string)
}
/*
+ * execute_worker_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_scan(ParallelScanStmt *parallelscan)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ PlannedStmt *planned_stmt;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+ BeginCommand("SELECT", DestNone);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ planned_stmt = create_worker_seqscan_plannedstmt(parallelscan);
+
+ if (parallelscan->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestRemoteBackend);
+ SetRemoteDestReceiverParams(receiver, NULL);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(planned_stmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelscan->params,
+ parallelscan->inst_options);
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelscan->inst_options)
+ memcpy(parallelscan->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelscan->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ /*
+ * Send appropriate CommandComplete to client. There is no
+ * need to send completion tag from worker as that won't be
+ * of any use considering the completiong tag of master backend
+ * will be used for sending to client.
+ */
+ EndCommand("", DestRemoteBackend);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index de988ba..b348bad 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -622,6 +622,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2437,6 +2439,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2624,6 +2636,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index b053659..784cfe0 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -287,6 +287,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -497,6 +500,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 0685e64..9d3d5e5 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -47,6 +47,8 @@ typedef struct ParallelContext
extern bool ParallelMessagePending;
extern int ParallelWorkerNumber;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExternalFunction(char *library_name, char *function_name, int nworkers);
extern void InitializeParallelDSM(ParallelContext *);
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index f459020..7a7bf75 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -115,4 +115,13 @@ typedef struct SysScanDescData
Snapshot snapshot; /* snapshot to unregister at end of scan */
} SysScanDescData;
+/* struct for scanning shared memory queues */
+typedef struct ShmScanDescData
+{
+ /* scan current state */
+ int num_shm_queues; /* number of shared memory queues used in scan. */
+ int ss_cqueue; /* current queue # in scan, if any */
+ bool shmscan_inited; /* false = scan not init'd yet */
+} ShmScanDescData;
+
#endif /* RELSCAN_H */
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..f3668ae
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,43 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ FmgrInfo *receive_functions;
+ Oid *typioparams;
+ HeapTuple tuple;
+ int num_shm_queues;
+ bool *queue_detached;
+ bool all_queues_detached;
+ bool all_heap_fetched;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+typedef struct ShmScanDescData *ShmScanDesc;
+
+extern worker_result ExecInitWorkerResult(TupleDesc tupdesc, int nWorkers);
+extern ShmScanDesc shm_beginscan(int num_queues);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, ShmScanDesc shmScan,
+ worker_result resultState, shm_mq_handle **responseq,
+ TupleDesc tupdesc, ScanDirection direction, bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..e8522fe 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,6 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeParallelSeqscan.h b/src/include/executor/nodeParallelSeqscan.h
new file mode 100644
index 0000000..b638a24
--- /dev/null
+++ b/src/include/executor/nodeParallelSeqscan.h
@@ -0,0 +1,33 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeparallelSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeParallelSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARALLELSEQSCAN_H
+#define NODEPARALLELSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern ParallelSeqScanState *ExecInitParallelSeqScan(ParallelSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecParallelSeqScan(ParallelSeqScanState *node);
+extern void ExecEndParallelSeqScan(ParallelSeqScanState *node);
+
+extern Size EstimateScanRelationIdSpace(Oid relId);
+extern void SerializeScanRelationId(Oid relId, Size maxsize,
+ char *start_address);
+extern void RestoreScanRelationId(Oid *relId, char *start_address);
+
+extern Size EstimateTargetListSpace(List *targetList);
+extern void SerializeTargetList(List *targetList, Size maxsize,
+ char *start_address);
+extern void RestoreTargetList(List **targetList, char *start_address);
+
+#endif /* NODEPARALLELSEQSCAN_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index ad7589d..067edbe 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -19,6 +19,13 @@
extern void pq_redirect_to_shm_mq(shm_mq *, shm_mq_handle *);
extern void pq_set_parallel_master(pid_t pid, BackendId backend_id);
+extern int
+mq_putmessage_direct(char msgtype, const char *s, size_t len);
+extern void
+pq_redirect_to_tuple_shm_mq(shm_mq_handle *mqh);
+extern bool
+is_tuple_shm_mq_enabled(void);
+
extern void pq_parse_errornotice(StringInfo str, ErrorData *edata);
#endif /* PQMQ_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 41288ed..844a9eb 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,12 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -1212,6 +1215,24 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * ParallelScanState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct ParallelScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ ShmScanDesc pss_currentShmScanDesc;
+ worker_result pss_workerResult;
+ char *inst_options_space;
+} ParallelScanState;
+
+typedef ParallelScanState ParallelSeqScanState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..b6f1493 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,7 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_ParallelSeqScan,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +98,7 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_ParallelSeqScanState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +219,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_ParallelSeqPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index 5b096c5..eb8c86a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index b1dfa85..929937d 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,9 +20,14 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
+#include "nodes/params.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "nodes/params.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +161,19 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for execution. */
+typedef struct ParallelScanStmt
+{
+ Index scanrelId;
+ List *targetList;
+ List *qual;
+ List *rangetableList;
+ ParamListInfo params;
+ shm_toc *toc;
+ uint64 shm_toc_scan_key;
+ int inst_options;
+ char *instrument;
+} ParallelScanStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 316c9ce..2ae52dd 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,8 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
#include "utils/lockwaitpolicy.h"
@@ -278,6 +280,23 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct ParallelSeqScan
+{
+ Scan scan;
+ /*
+ * Non-zero values of toc and shm_toc_key indicates that this
+ * node will be used for execution of parallel scan in worker
+ * backend.
+ */
+ shm_toc *toc;
+ uint64 shm_toc_key;
+ int num_workers;
+} ParallelSeqScan;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..c5eb319 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,12 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct ParallelSeqPath
+{
+ Path path;
+ int num_workers;
+} ParallelSeqPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..0b6a469 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_parallelseqscan(ParallelSeqPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..32c3e0d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,8 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern ParallelSeqPath *create_parallelseqscan_path(PlannerInfo *root,
+ RelOptInfo *rel, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 082f7d7..eb6be5a 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -41,6 +41,8 @@ extern Plan *optimize_minmax_aggregates(PlannerInfo *root, List *tlist,
* prototypes for plan/createplan.c
*/
extern Plan *create_plan(PlannerInfo *root, Path *best_path);
+extern SeqScan *
+create_worker_seqscan_plan(ParallelScanStmt *parallelscan);
extern SubqueryScan *make_subqueryscan(List *qptlist, List *qpqual,
Index scanrelid, Plan *subplan);
extern ForeignScan *make_foreignscan(List *qptlist, List *qpqual,
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..c2aa875 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_seqscan_plannedstmt(ParallelScanStmt *parallelscan);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..6d0b590
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,32 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Index scanrelId, List *targetList,
+ List *qual, EState *estate,
+ Relation rel, char **inst_options_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..dd176b5 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -89,6 +89,7 @@ typedef enum
DestDebug, /* results go to debugging output */
DestRemote, /* results sent to frontend process */
DestRemoteExecute, /* sent to frontend, in Execute command */
+ DestRemoteBackend, /* parallel worker send results to master backend */
DestSPI, /* results sent to SPI manager */
DestTuplestore, /* results sent to Tuplestore */
DestIntoRel, /* results sent to relation (SELECT INTO) */
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 3e17770..9eebc51 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_scan(ParallelScanStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Feb 17, 2015 at 9:52 PM, Andres Freund <andres@2ndquadrant.com>
wrote:
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 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 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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 xyzWhat 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.xNow, 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.xThe 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.xNow 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 = 1Here, 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 = 1That'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>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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]/messages/by-id/CA+TgmobCMwFOz-9=hFv=hJ4SH7p=5X6Ga5V=WtT8=huzE6C+Mg@mail.gmail.com + parallel-heap-scan.patch[2]/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
[1]: /messages/by-id/CA+TgmobCMwFOz-9=hFv=hJ4SH7p=5X6Ga5V=WtT8=huzE6C+Mg@mail.gmail.com
/messages/by-id/CA+TgmobCMwFOz-9=hFv=hJ4SH7p=5X6Ga5V=WtT8=huzE6C+Mg@mail.gmail.com
[2]: /messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v8.patchapplication/octet-stream; name=parallel_seqscan_v8.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..d8bd596
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,92 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+//#include "access/tupdesc.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+
+/*
+ * ExecInitWorkerResult
+ *
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(void)
+{
+ worker_result workerResult;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+
+ return workerResult;
+}
+
+/*
+ * shm_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, worker_result resultState,
+ TupleQueueFunnel *funnel, ScanDirection direction,
+ bool *fromheap)
+{
+ HeapTuple tup;
+
+ while (!resultState->all_workers_done || !resultState->local_scan_done)
+ {
+ if (!resultState->all_workers_done)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnel, !resultState->local_scan_done,
+ &resultState->all_workers_done);
+ if (HeapTupleIsValid(tup))
+ {
+ *fromheap = false;
+ return tup;
+ }
+ }
+ if (!resultState->local_scan_done)
+ {
+ tup = heap_getnext(scanDesc, direction);
+ if (HeapTupleIsValid(tup))
+ {
+ *fromheap = true;
+ return tup;
+ }
+ resultState->local_scan_done = true;
+ }
+ }
+
+ return NULL;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index a951c55..8410afa 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +917,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1069,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1211,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1331,6 +1354,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2214,6 +2245,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index 1c8be25..f13b7bc 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 07526e8..9a3e285 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..caf9855 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,13 +191,20 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification or projection as for them the same is done
+ * by worker backend. This case will happen only for parallel query
+ * where we push down the qualification and projection (targetlist)
+ * information.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
*/
- if (projInfo)
+ if (projInfo && slot->tts_fromheap)
{
/*
* Form a projection tuple, store it in the result tuple slot
@@ -211,6 +218,23 @@ ExecScan(ScanState *node,
return resultSlot;
}
}
+ else if (projInfo && !slot->tts_fromheap)
+ {
+ /*
+ * Store the tuple we got from shared memory tuple queue
+ * in projection slot as the worker backend wtakes care
+ * of doing projection. We don't need to free this tuple
+ * as this is pointing to scan tuple slot which will take
+ * care of freeing it.
+ */
+ ExecStoreTuple(econtext->ecxt_scantuple->tts_tuple, /* tuple to store */
+ projInfo->pi_slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ false); /* pfree this pointer */
+
+ return projInfo->pi_slot;
+ }
else
{
/*
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..56e509d 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -21,6 +21,8 @@ BufferUsage pgBufferUsage;
static void BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add, const BufferUsage *sub);
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
/* Allocate new instrumentation structure(s) */
@@ -127,6 +129,28 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
static void
BufferUsageAccumDiff(BufferUsage *dst,
@@ -148,3 +172,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..71f6daa
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,301 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * FunnelNext retrieve next tuple from either heap or shared memory segment.
+ * ExecInitFunnel creates and initializes a parallel seqscan node.
+ * ExecEndFunnel releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * FunnelNext
+ *
+ * This is a workhorse for ExecFunnel
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+FunnelNext(FunnelState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc,
+ node->pss_workerResult,
+ node->funnel,
+ direction,
+ &fromheap);
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ fromheap ? scandesc->rs_cbuf : InvalidBuffer, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * FunnelRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+FunnelRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, Funnel never use keys in
+ * heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitFunnelRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnelRelation(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ &pscan,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+
+ node->ss.ss_currentRelation = currentRelation;
+ node->ss.ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnelRelation(funnelstate, estate, eflags);
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ funnelstate->pss_workerResult = ExecInitWorkerResult();
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+
+ /*
+ * if parallel context is set and workers are not
+ * registered, register them now.
+ */
+ if (node->pcxt && !node->fs_workersReady)
+ {
+ /* Register backend workers. */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ }
+
+ node->fs_workersReady = true;
+ }
+
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) FunnelNext,
+ (ExecScanRecheckMtd) FunnelRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt)
+ {
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..fb4efa3
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,259 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss_currentScanDesc;
+ estate = node->ps.state;
+ direction = estate->es_direction;
+ slot = node->ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory.
+ */
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ps.plan = (Plan *) node;
+ scanstate->ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ps);
+ ExecInitScanTupleSlot(estate, scanstate);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ps);
+ ExecAssignScanProjectionInfo(scanstate);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss_currentRelation;
+ scanDesc = node->ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..ee4e03e
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,272 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+ memcpy(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9fe8008..e51fc38 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -354,6 +354,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4044,6 +4081,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 775f482..3382ab2 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -439,6 +439,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2886,6 +2904,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..aa278c5 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,187 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ /* sizeof(ParamListInfoData) includes the first array element */
+ size = sizeof(ParamListInfoData) +
+ (num_params - 1) * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..2bae475 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,91 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1494,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 78ef229..5f5980f 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -219,6 +227,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..3149247
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,121 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /*
+ * parallel scan is not supported for joins.
+ */
+ if (root->simple_rel_array_size > 2)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 76ba1bf..744e652 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_Funnel:
+ case T_PartialSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,87 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3318,6 +3428,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b02a107..182c70d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,50 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_worker_scan_plannedstmt(PartialSeqScan *partialscan, List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..1b63f23 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel scan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 5a1d539..8ea91ec 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2163,6 +2163,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..c1ffe78 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..0c38e60
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,410 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/dbcommands.h"
+#include "commands/async.h"
+#include "executor/nodeFunnel.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "storage/ipc.h"
+#include "storage/procsignal.h"
+#include "storage/procarray.h"
+#include "storage/shm_toc.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/builtins.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, ParallelHeapScanDesc *pscan,
+ Size plannedstmt_size, Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, ParallelHeapScanDesc *pscan,
+ Size plannedstmt_size, Size pscan_size)
+{
+ char *plannedstmtdata;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ *pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(*pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, *pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_worker_scan_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ pscan, plannedstmt_size, pscan_size);
+
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options, &instrument);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index ac431e5..4c303dd 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index ea2a432..17f322f 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1191,6 +1193,80 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 791543e..abc2b8f 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2537,6 +2539,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2724,6 +2736,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index f8f9ce1..fbe6042 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -290,6 +290,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -500,6 +503,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 0685e64..9d3d5e5 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -47,6 +47,8 @@ typedef struct ParallelContext
extern bool ParallelMessagePending;
extern int ParallelWorkerNumber;
+extern int ParallelWorkerNumber;
+
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
extern ParallelContext *CreateParallelContextForExternalFunction(char *library_name, char *function_name, int nworkers);
extern void InitializeParallelDSM(ParallelContext *);
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..80d06ac
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,36 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "executor/tqueue.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ bool all_workers_done;
+ bool local_scan_done;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+extern worker_result ExecInitWorkerResult(void);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, worker_result resultState,
+ TupleQueueFunnel *funnel, ScanDirection direction,
+ bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..e8522fe 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,6 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..df7e11e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodefunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..f02bcca
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,23 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 59b17f3..323b35b 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,9 +16,13 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
+#include "storage/shm_mq.h"
#include "utils/reltrigger.h"
#include "utils/sortsupport.h"
#include "utils/tuplestore.h"
@@ -389,6 +393,12 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1213,6 +1223,29 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScan uses a bare SeqScanState as its state node, since
+ * it needs no additional fields.
+ */
+typedef SeqScanState PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ worker_result pss_workerResult;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ bool fs_workersReady;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..6acbe67 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index ac13302..ea8e240 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,9 +20,16 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "nodes/params.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +163,16 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f6683f0..8099f78 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,8 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
#include "utils/lockwaitpolicy.h"
@@ -279,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..df1ab5e 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -103,6 +103,8 @@ typedef struct PlannerGlobal
bool hasRowSecurity; /* row security applied? */
+ bool parallelModeNeeded; /* parallel plans need parallelmode */
+
} PlannerGlobal;
/* macro for fetching the Plan associated with a SubPlan node */
@@ -737,6 +739,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..3b7ed92 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_scan_plannedstmt(PartialSeqScan *partialscan, List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..1d05d79
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,39 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_INST_OPTIONS 2
+#define PARALLEL_KEY_INST_INFO 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+#define PARALLEL_KEY_SCAN 5
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 3e17770..489af46 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
On Wed, Mar 4, 2015 at 6:17 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
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]
/messages/by-id/CA+TgmobCMwFOz-9=hFv=hJ4SH7p=5X6Ga5V=WtT8=huzE6C+Mg@mail.gmail.com
[2]
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
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.
It is based on commit-id:5a2a48f0 and parallel-mode-v7.patch
and parallel-heap-scan.patch
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v9.patchapplication/octet-stream; name=parallel_seqscan_v9.patchDownload
diff --git a/src/backend/access/Makefile b/src/backend/access/Makefile
index 21721b4..823d5c3 100644
--- a/src/backend/access/Makefile
+++ b/src/backend/access/Makefile
@@ -8,6 +8,6 @@ subdir = src/backend/access
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc spgist transam
+SUBDIRS = brin common gin gist hash heap index nbtree rmgrdesc shmmq spgist transam
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 383e15b..d384e8f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1582,6 +1582,20 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
}
/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/access/shmmq/Makefile b/src/backend/access/shmmq/Makefile
new file mode 100644
index 0000000..aeae8d9
--- /dev/null
+++ b/src/backend/access/shmmq/Makefile
@@ -0,0 +1,17 @@
+#-------------------------------------------------------------------------
+#
+# Makefile--
+# Makefile for access/shmmq
+#
+# IDENTIFICATION
+# src/backend/access/shmmq/Makefile
+#
+#-------------------------------------------------------------------------
+
+subdir = src/backend/access/shmmq
+top_builddir = ../../../..
+include $(top_builddir)/src/Makefile.global
+
+OBJS = shmmqam.o
+
+include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/access/shmmq/shmmqam.c b/src/backend/access/shmmq/shmmqam.c
new file mode 100644
index 0000000..9c57580
--- /dev/null
+++ b/src/backend/access/shmmq/shmmqam.c
@@ -0,0 +1,91 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.c
+ * shared memory queue access method code
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/access/shmmq/shmmqam.c
+ *
+ *
+ * INTERFACE ROUTINES
+ * shm_getnext - retrieve next tuple in queue
+ *
+ * NOTES
+ * This file contains the shmmq_ routines which implement
+ * the POSTGRES shared memory access method used for all POSTGRES
+ * relations.
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/htup.h"
+#include "access/htup_details.h"
+#include "access/shmmqam.h"
+#include "fmgr.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "utils/lsyscache.h"
+
+
+
+/*
+ * ExecInitWorkerResult
+ *
+ * Initializes the result state to retrieve tuples from worker backends.
+ */
+worker_result
+ExecInitWorkerResult(void)
+{
+ worker_result workerResult;
+
+ workerResult = palloc0(sizeof(worker_result_state));
+
+ return workerResult;
+}
+
+/*
+ * shm_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in parallel sequential scan.
+ */
+HeapTuple
+shm_getnext(HeapScanDesc scanDesc, worker_result resultState,
+ TupleQueueFunnel *funnel, ScanDirection direction,
+ bool *fromheap)
+{
+ HeapTuple tup;
+
+ while (!resultState->all_workers_done || !resultState->local_scan_done)
+ {
+ if (!resultState->all_workers_done)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnel, !resultState->local_scan_done,
+ &resultState->all_workers_done);
+ if (HeapTupleIsValid(tup))
+ {
+ *fromheap = false;
+ return tup;
+ }
+ }
+ if (!resultState->local_scan_done)
+ {
+ tup = heap_getnext(scanDesc, direction);
+ if (HeapTupleIsValid(tup))
+ {
+ *fromheap = true;
+ return tup;
+ }
+ resultState->local_scan_done = true;
+ }
+ }
+
+ return NULL;
+}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index a951c55..8410afa 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +917,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1069,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1211,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1331,6 +1354,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2214,6 +2245,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..268ee3f 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -155,6 +156,10 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +463,11 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ /* Backward scan is not suppotted for parallel sequiantel scan. */
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index 1c8be25..f13b7bcb 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 07526e8..9a3e285 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execScan.c b/src/backend/executor/execScan.c
index 3f0d809..7916ea3 100644
--- a/src/backend/executor/execScan.c
+++ b/src/backend/executor/execScan.c
@@ -191,13 +191,20 @@ ExecScan(ScanState *node,
* check for non-nil qual here to avoid a function call to ExecQual()
* when the qual is nil ... saves only a few cycles, but they add up
* ...
+ *
+ * check for non-heap tuples (can get such tuples from shared memory
+ * message queue's in case of parallel query), for such tuples no need
+ * to perform qualification or projection as for them the same is done
+ * by worker backend. This case will happen only for parallel query
+ * where we push down the qualification and projection (targetlist)
+ * information.
*/
- if (!qual || ExecQual(qual, econtext, false))
+ if (!slot->tts_fromheap || !qual || ExecQual(qual, econtext, false))
{
/*
* Found a satisfactory scan tuple.
*/
- if (projInfo)
+ if (projInfo && slot->tts_fromheap)
{
/*
* Form a projection tuple, store it in the result tuple slot
@@ -211,6 +218,23 @@ ExecScan(ScanState *node,
return resultSlot;
}
}
+ else if (projInfo && !slot->tts_fromheap)
+ {
+ /*
+ * Store the tuple we got from shared memory tuple queue
+ * in projection slot as the worker backend takes care
+ * of doing projection. We don't need to free this tuple
+ * as this is pointing to scan tuple slot which will take
+ * care of freeing it.
+ */
+ ExecStoreTuple(econtext->ecxt_scantuple->tts_tuple, /* tuple to store */
+ projInfo->pi_slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ false); /* pfree this pointer */
+
+ return projInfo->pi_slot;
+ }
else
{
/*
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..4c5bd88 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -123,6 +123,7 @@ MakeTupleTableSlot(void)
slot->tts_values = NULL;
slot->tts_isnull = NULL;
slot->tts_mintuple = NULL;
+ slot->tts_fromheap = true;
return slot;
}
@@ -473,6 +474,8 @@ ExecClearTuple(TupleTableSlot *slot) /* slot in which to store tuple */
slot->tts_isempty = true;
slot->tts_nvalid = 0;
+ slot->tts_fromheap = true;
+
return slot;
}
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..56e509d 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -21,6 +21,8 @@ BufferUsage pgBufferUsage;
static void BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add, const BufferUsage *sub);
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
/* Allocate new instrumentation structure(s) */
@@ -127,6 +129,28 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
static void
BufferUsageAccumDiff(BufferUsage *dst,
@@ -148,3 +172,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..74e1e44
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,423 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * FunnelNext retrieve next tuple from either heap or shared memory segment.
+ * ExecInitFunnel creates and initializes a parallel seqscan node.
+ * ExecEndFunnel releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * FunnelNext
+ *
+ * This is a workhorse for ExecFunnel
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+FunnelNext(FunnelState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+ bool fromheap = true;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table based on result tuple descriptor.
+ */
+ tuple = shm_getnext(scandesc,
+ node->pss_workerResult,
+ node->funnel,
+ direction,
+ &fromheap);
+
+ slot->tts_fromheap = fromheap;
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass '!fromheap'
+ * because tuples returned by shm_getnext() are either pointers that are
+ * created with palloc() or are pointers onto disk pages and so it should
+ * be pfree()'d accordingly. Note also that ExecStoreTuple will increment
+ * the refcount of the buffer; the refcount will not be dropped until the
+ * tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ fromheap ? scandesc->rs_cbuf : InvalidBuffer, /* buffer associated with this
+ * tuple */
+ !fromheap); /* pfree this pointer if not from heap */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * FunnelRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+FunnelRecheck(SeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, Funnel never use keys in
+ * heap_beginscan (and this is very bad) - so, here
+ * we do not check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitFunnelRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnelRelation(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * For Explain statement, we don't want to initialize workers as
+ * those are maily needed to execute the plan, however scan descriptor
+ * still needs to be initialized for the purpose of InitNode functionality
+ * and EndNode functionality assumes that scan descriptor and scan relation
+ * must be initialized, probably we can change that but that will make
+ * the code EndParallelSeqScan look different than other node's end
+ * functionality.
+ */
+ if (eflags & EXEC_FLAG_EXPLAIN_ONLY)
+ {
+ /* initialize a heapscan */
+ currentScanDesc = heap_beginscan(currentRelation,
+ estate->es_snapshot,
+ 0,
+ NULL);
+ }
+ else
+ {
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ &pscan,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+ }
+
+ node->ss.ss_currentRelation = currentRelation;
+ node->ss.ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnelRelation(funnelstate, estate, eflags);
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ funnelstate->pss_workerResult = ExecInitWorkerResult();
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+
+ /*
+ * if parallel context is set and workers are not
+ * registered, register them now.
+ */
+ if (node->pcxt && !node->fs_workersReady)
+ {
+ /* Register backend workers. */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ }
+
+ node->fs_workersReady = true;
+ }
+
+ return ExecScan((ScanState *) &node->ss,
+ (ExecScanAccessMtd) FunnelNext,
+ (ExecScanRecheckMtd) FunnelRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ if (node->pcxt && node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+ else if (node->pcxt)
+ {
+ int i;
+
+ /*
+ * We only need to free the memory allocated to initialize
+ * parallel workers as workers are still not started.
+ */
+ dlist_delete(&node->pcxt->node);
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(node->pcxt->worker[i].error_mqh);
+ node->pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /*
+ * If we have allocated a shared memory segment, detach it. This will
+ * implicitly detach the error queues, and any other shared memory
+ * queues, stored there.
+ */
+ if (node->pcxt->seg != NULL)
+ dsm_detach(node->pcxt->seg);
+
+ /* Free the worker array itself. */
+ pfree(node->pcxt->worker);
+ node->pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ HeapScanDesc scan;
+ ParallelHeapScanDesc pscan = NULL;
+ EState *estate = node->ss.ps.state;
+
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ &pscan,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ node->fs_workersReady = false;
+
+ node->pss_workerResult->all_workers_done = 0;
+ node->pss_workerResult->local_scan_done = 0;
+ }
+
+ scan = node->ss.ss_currentScanDesc;
+
+ heap_parallel_rescan(pscan, /* scan desc */
+ scan); /* new scan keys */
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..fb4efa3
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,259 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/shmmqam.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss_currentScanDesc;
+ estate = node->ps.state;
+ direction = estate->es_direction;
+ slot = node->ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory.
+ */
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ps.plan = (Plan *) node;
+ scanstate->ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ps);
+ ExecInitScanTupleSlot(estate, scanstate);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ps);
+ ExecAssignScanProjectionInfo(scanstate);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss_currentRelation;
+ scanDesc = node->ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..ee4e03e
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,272 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+ memcpy(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 9fe8008..e51fc38 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -354,6 +354,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4044,6 +4081,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 775f482..3382ab2 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -439,6 +439,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2886,6 +2904,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..aa278c5 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,187 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ /* sizeof(ParamListInfoData) includes the first array element */
+ size = sizeof(ParamListInfoData) +
+ (num_params - 1) * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..2bae475 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,91 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1494,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 5a9daf0..282e5ff 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..0b25b39
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,115 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..9f084ab 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,87 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3431,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b02a107..182c70d 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,50 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_worker_scan_plannedstmt(PartialSeqScan *partialscan, List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..ef8c317 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 5a1d539..8ea91ec 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2163,6 +2163,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..c1ffe78 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..28705d6
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,400 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/nodeFunnel.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, ParallelHeapScanDesc *pscan,
+ Size plannedstmt_size, Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, ParallelHeapScanDesc *pscan,
+ Size plannedstmt_size, Size pscan_size)
+{
+ char *plannedstmtdata;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ *pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(*pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, *pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan, int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_worker_scan_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ pscan, plannedstmt_size, pscan_size);
+
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options, &instrument);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index ac431e5..4c303dd 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index ea2a432..17f322f 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1191,6 +1193,80 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 791543e..abc2b8f 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2537,6 +2539,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2724,6 +2736,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index f8f9ce1..fbe6042 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -290,6 +290,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -500,6 +503,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index fb2b5f0..d4f4e2d 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -117,6 +117,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
diff --git a/src/include/access/shmmqam.h b/src/include/access/shmmqam.h
new file mode 100644
index 0000000..80d06ac
--- /dev/null
+++ b/src/include/access/shmmqam.h
@@ -0,0 +1,36 @@
+/*-------------------------------------------------------------------------
+ *
+ * shmmqam.h
+ * POSTGRES shared memory queue access method definitions.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/shmmqam.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef SHMMQAM_H
+#define SHMMQAM_H
+
+#include "access/relscan.h"
+#include "executor/tqueue.h"
+#include "libpq/pqmq.h"
+
+
+/* Private state maintained across calls to shm_getnext. */
+typedef struct worker_result_state
+{
+ bool all_workers_done;
+ bool local_scan_done;
+} worker_result_state;
+
+typedef struct worker_result_state *worker_result;
+
+extern worker_result ExecInitWorkerResult(void);
+extern HeapTuple shm_getnext(HeapScanDesc scanDesc, worker_result resultState,
+ TupleQueueFunnel *funnel, ScanDirection direction,
+ bool *fromheap);
+
+#endif /* SHMMQAM_H */
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..e8522fe 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,6 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..7c6d93f
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodefunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..f02bcca
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,23 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/executor/tuptable.h b/src/include/executor/tuptable.h
index 48f84bf..e5dec1e 100644
--- a/src/include/executor/tuptable.h
+++ b/src/include/executor/tuptable.h
@@ -127,6 +127,8 @@ typedef struct TupleTableSlot
MinimalTuple tts_mintuple; /* minimal tuple, or NULL if none */
HeapTupleData tts_minhdr; /* workspace for minimal-tuple-only case */
long tts_off; /* saved state for slot_deform_tuple */
+ bool tts_fromheap; /* indicates whether the tuple is fetched from
+ heap or shrared memory message queue */
} TupleTableSlot;
#define TTS_HAS_PHYSICAL_TUPLE(slot) \
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 59b17f3..32e3baf 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,10 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
+#include "access/shmmqam.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +392,12 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1213,6 +1222,29 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScan uses a bare SeqScanState as its state node, since
+ * it needs no additional fields.
+ */
+typedef SeqScanState PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * dsm_segment dynamic shared memory segment to setup worker queues
+ * responseq shared memory queues to receive data from workers
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ worker_result pss_workerResult;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ bool fs_workersReady;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 97ef0fc..6acbe67 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index ac13302..ea8e240 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,9 +20,16 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "nodes/params.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +163,16 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f6683f0..8099f78 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,8 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
#include "utils/lockwaitpolicy.h"
@@ -279,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..df1ab5e 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -103,6 +103,8 @@ typedef struct PlannerGlobal
bool hasRowSecurity; /* row security applied? */
+ bool parallelModeNeeded; /* parallel plans need parallelmode */
+
} PlannerGlobal;
/* macro for fetching the Plan associated with a SubPlan node */
@@ -737,6 +739,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..3b7ed92 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *
+create_worker_scan_plannedstmt(PartialSeqScan *partialscan, List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..1d05d79
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,39 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_INST_OPTIONS 2
+#define PARALLEL_KEY_INST_INFO 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+#define PARALLEL_KEY_SCAN 5
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ ParallelHeapScanDesc *pscan,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 3e17770..489af46 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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?
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, 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.
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?
I think stopping/terminating workers should handle such a case.
Thanks for pointing out this case, I will change it in next update.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 10-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]/messages/by-id/CA+TgmobM7X6jgre442638b+33h1EWa=vcZqnsvzEdX057ZHVuw@mail.gmail.com, 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]: /messages/by-id/CA+TgmobM7X6jgre442638b+33h1EWa=vcZqnsvzEdX057ZHVuw@mail.gmail.com
/messages/by-id/CA+TgmobM7X6jgre442638b+33h1EWa=vcZqnsvzEdX057ZHVuw@mail.gmail.com
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Mar 11, 2015 at 1:01 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
I have tried, but there are couple of failures while applying latest
access-parallel-safety patch, so left it as it is for now.
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.
Agreed. Moved the code/logic to nodeFunnel.c
- 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.
Agreed and made the required changes.
The logic
in InitFunnelRelation() belongs in the parallel seq scan node, not the
funnel.
I think we should retain initialization of parallelcontext in InitFunnel().
Apart from that, I have moved other stuff to partial seq scan node.
ExecReScanFunnel() cannot be calling heap_parallel_rescan();
it needs to *not know* that there is a parallel scan under it.
Agreed. I think it is better to be do that as part of partial seq scan
node.
The
comment in FunnelRecheck is a copy-and-paste from elsewhere that is
not applicable to a generic funnel mode.
With new changes, this API is not required.
- 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?
Agreed and changed as per suggestion.
- 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.
Sure, removed (tts_fromheap becomes redundant with new changes).
- 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.
Yes, but isn't other nodes also work this way, example join node will
display the accumulated stats for buffer usage, but for timing, it will
just use the time for that node (which automatically includes some
part of execution of child nodes, but it is not direct accumulation)?
We might need to think carefully about what it makes sense
to display in the EXPLAIN output in parallel cases.
Currently the Explain for parallel scan on relation will display the
Funnel node which contains aggregated stat of all workers and the
number of workers and Partial Seq Scan node containing stats for
the scan done by master backend. Do we want to display something
more?
Current result of Explain statement is as below:
postgres=# explain (analyze,buffers) select c1 from t1 where c1 > 90000;
QUERY PLAN
--------------------------------------------------------------------------------
-------------------------------------------
Funnel on t1 (cost=0.00..43750.44 rows=9905 width=4) (actual
time=1097.236..15
30.416 rows=10000 loops=1)
Filter: (c1 > 90000)
Rows Removed by Filter: 65871
Number of Workers: 2
Buffers: shared hit=96 read=99905
-> Partial Seq Scan on t1 (cost=0.00..101251.01 rows=9905 width=4)
(actual
time=1096.188..1521.810 rows=2342 loops=1)
Filter: (c1 > 90000)
Rows Removed by Filter: 24130
Buffers: shared hit=33 read=26439
Planning time: 0.143 ms
Execution time: 1533.438 ms
(11 rows)
- The header comment on nodeFunnel.h capitalizes the filename incorrectly.
Changed.
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).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v10.patchapplication/octet-stream; name=parallel_seqscan_v10.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 383e15b..d384e8f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1582,6 +1582,20 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
}
/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 8d39bf2..d2817b2 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -535,6 +535,8 @@ HandleParallelMessageInterrupt(void)
InterruptPending = true;
ParallelMessagePending = true;
+ SetLatch(MyLatch);
+
errno = save_errno;
}
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index a951c55..f8acfc8 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +918,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1073,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1216,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1322,6 +1350,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1331,6 +1360,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2214,6 +2251,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..10dc319 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -155,6 +157,14 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +468,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index 1c8be25..f13b7bcb 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 07526e8..9a3e285 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..56e509d 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -21,6 +21,8 @@ BufferUsage pgBufferUsage;
static void BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add, const BufferUsage *sub);
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
/* Allocate new instrumentation structure(s) */
@@ -127,6 +129,28 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
static void
BufferUsageAccumDiff(BufferUsage *dst,
@@ -148,3 +172,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+static void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..23f0245
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,376 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ estate->toc = node->pcxt->toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ /* Initialize scan state of workers. */
+ funnelstate->all_workers_done = false;
+ funnelstate->local_scan_done = false;
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+
+ /*
+ * if parallel context is set and workers are not
+ * registered, register them now.
+ */
+ if (node->pcxt && !node->fs_workersReady)
+ {
+ /* Register backend workers. */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ }
+
+ node->fs_workersReady = true;
+ }
+
+ return funnel_getnext(node);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ if (node->pcxt && node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ ExitParallelMode();
+ }
+ else if (node->pcxt)
+ {
+ int i;
+
+ /*
+ * We only need to free the memory allocated to initialize
+ * parallel workers as workers are still not started.
+ */
+ dlist_delete(&node->pcxt->node);
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(node->pcxt->worker[i].error_mqh);
+ node->pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /*
+ * If we have allocated a shared memory segment, detach it. This will
+ * implicitly detach the error queues, and any other shared memory
+ * queues, stored there.
+ */
+ if (node->pcxt->seg != NULL)
+ dsm_detach(node->pcxt->seg);
+
+ /* Free the worker array itself. */
+ pfree(node->pcxt->worker);
+ node->pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(node->pcxt);
+
+ ExitParallelMode();
+ }
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ EState *estate = node->ss.ps.state;
+
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+
+ estate->toc = node->pcxt->toc;
+
+ ExecReScan(node->ss.ps.lefttree);
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues, it does fetch the data from local
+ * node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while (!funnelstate->all_workers_done || !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..55aa266
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,288 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss_currentScanDesc;
+ estate = node->ps.state;
+ direction = estate->es_direction;
+ slot = node->ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory.
+ */
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ps.plan = (Plan *) node;
+ scanstate->ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ps);
+ ExecInitScanTupleSlot(estate, scanstate);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ps);
+ ExecAssignScanProjectionInfo(scanstate);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss_currentRelation;
+ scanDesc = node->ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ HeapScanDesc scan;
+ ParallelHeapScanDesc pscan;
+ EState *estate = node->ps.state;
+
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ scan = node->ss_currentScanDesc;
+
+ heap_parallel_rescan(pscan, /* scan desc */
+ scan); /* new scan keys */
+
+ ExecScanReScan((ScanState *) node);
+}
\ No newline at end of file
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..ee4e03e
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,272 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+ memcpy(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ebb6f3a..c1d77e0 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -354,6 +354,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4047,6 +4084,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 775f482..3382ab2 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -439,6 +439,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2886,6 +2904,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..aa278c5 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,187 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ /* sizeof(ParamListInfoData) includes the first array element */
+ size = sizeof(ParamListInfoData) +
+ (num_params - 1) * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..2bae475 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,91 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1494,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 5a9daf0..282e5ff 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..0b25b39
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,115 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * check_simple_qual -
+ * Check if qual is made only of simple things we can
+ * hand out directly to backend worker for execution.
+ *
+ * XXX - Currently we don't allow to push an expression
+ * if it contains volatile function, however eventually we
+ * need a mechanism (proisparallel) with which we can distinquish
+ * the functions that can be pushed for execution by parallel
+ * worker.
+ */
+static bool
+check_simple_qual(Node *node)
+{
+ if (node == NULL)
+ return TRUE;
+
+ if (contain_volatile_functions(node))
+ return FALSE;
+
+ return TRUE;
+}
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0.
+ */
+ if (parallel_seqscan_degree <= 0)
+ return;
+
+ /* parallel scan is supportted only for Select statements. */
+ if (root->parse->commandType != CMD_SELECT)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * parallel scan is not supported for mutable functions
+ */
+ if (!check_simple_qual((Node*) extract_actual_clauses(rel->baserestrictinfo, false)))
+ return;
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..32cefe6 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,84 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3428,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index b02a107..590d0df 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -260,6 +260,51 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..ef8c317 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 5a1d539..8ea91ec 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2163,6 +2163,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 1395a21..c1ffe78 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..b3eb876
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,401 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/nodeFunnel.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+ ParallelHeapScanDesc pscan;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ int nWorkers)
+{
+ bool already_in_parallel_mode = IsInParallelMode();
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options, &instrument);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index ac431e5..4c303dd 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index ea2a432..17f322f 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1191,6 +1193,80 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 791543e..abc2b8f 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2537,6 +2539,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2724,6 +2736,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index f8f9ce1..fbe6042 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -290,6 +290,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -500,6 +503,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index fb2b5f0..d4f4e2d 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -117,6 +117,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..e8522fe 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,6 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..3af3a0e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..cb05be7
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 59b17f3..93eab5d 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +391,12 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1213,6 +1221,37 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScan uses a bare SeqScanState as its state node, since
+ * it needs no additional fields.
+ */
+typedef SeqScanState PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to retrieve instrumentation information.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 38469ef..3f3d572 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 497559d..3f113b1 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,9 +20,16 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "nodes/params.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
#include "utils/lockwaitpolicy.h"
/* Possible sources of a Query */
@@ -156,6 +163,16 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index f6683f0..8099f78 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -18,6 +18,8 @@
#include "lib/stringinfo.h"
#include "nodes/bitmapset.h"
#include "nodes/primnodes.h"
+#include "storage/block.h"
+#include "storage/shm_toc.h"
#include "utils/lockwaitpolicy.h"
@@ -279,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 6845a40..21357be 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -737,6 +737,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..7bc7d7e 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..fe428eb
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,38 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_INST_OPTIONS 2
+#define PARALLEL_KEY_INST_INFO 3
+#define PARALLEL_KEY_TUPLE_QUEUE 4
+#define PARALLEL_KEY_SCAN 5
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 3e17770..489af46 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
/messages/by-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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.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.
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.comPlease 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.
--
Thom
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.comPlease 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 pgbenchCREATE 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 segmentsAnd 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 startedI 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 12, 2015 at 4:22 PM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp>
wrote:
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
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
Attachments:
TupleQueueFunnelNext-bugfix.patchtext/x-diff; name=TupleQueueFunnelNext-bugfix.patchDownload
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
index ee4e03e..8a6c6f3 100644
--- a/src/backend/executor/tqueue.c
+++ b/src/backend/executor/tqueue.c
@@ -234,6 +234,8 @@ TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
&funnel->queue[funnel->nextqueue + 1],
sizeof(shm_mq_handle *)
* (funnel->nqueues - funnel->nextqueue));
+
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
if (funnel->nextqueue < waitpos)
--waitpos;
}
@@ -260,7 +262,7 @@ TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
* and return NULL (if we're in non-blocking mode) or wait for the
* process latch to be set (otherwise).
*/
- if (funnel->nextqueue == waitpos)
+ if (result != SHM_MQ_DETACHED && funnel->nextqueue == waitpos)
{
if (nowait)
return NULL;
On Fri, Mar 13, 2015 at 2:12 PM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp>
wrote:
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.
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.
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.
Thanks.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
fix_tupqueue_issue_v1.patchapplication/octet-stream; name=fix_tupqueue_issue_v1.patchDownload
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
index ee4e03e..8e7f35e 100644
--- a/src/backend/executor/tqueue.c
+++ b/src/backend/executor/tqueue.c
@@ -230,10 +230,13 @@ TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
*done = true;
return NULL;
}
- memcpy(&funnel->queue[funnel->nextqueue],
- &funnel->queue[funnel->nextqueue + 1],
- sizeof(shm_mq_handle *)
- * (funnel->nqueues - funnel->nextqueue));
+ if (funnel->nextqueue <= funnel->nqueues)
+ memcpy(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+ else
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
if (funnel->nextqueue < waitpos)
--waitpos;
}
On Thu, Mar 12, 2015 at 10:35 PM, Thom Brown <thom@linux.com> wrote:
Another problem. I restarted the instance (just in case), and get this
error:
# \df+ *.*
ERROR: cannot retain locks acquired while in parallel mode
This problem occurs because above statement is trying to
execute parallel_unsafe function (obj_description) in parallelmode.
This will be resolved once parallel_seqscan patch is integrated
with access-parallel-safety patch [1]https://commitfest.postgresql.org/4/155/.
[1]: https://commitfest.postgresql.org/4/155/
https://commitfest.postgresql.org/4/155/
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Mar 10, 2015 at 12:26 PM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
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
it can so happen that by that time some of the workers are still not
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Mar 12, 2015 at 9:50 PM, Thom Brown <thom@linux.com> wrote:
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:
Thanks for looking into patch.
So as per this report, I am seeing 3 different problems in it.
Problem-1:
---------------------
# 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
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.
One way to fix could be that we increase the number of dsm segments
that can be created in a system/backend, but it seems to me that in
reality there might not be many such plans which would need so many
dsm segments, unless user tinkers too much with costing and even if
he does, he can increase max_connections to avoid such problem.
I would like to see opinion of other people on this matter.
Problem-2:
--------------------
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
This seems to be unrelated to first issue (as the statement in log has
nothing to do with Problem-1) and this could be same issue what
Amit Langote has reported, so we can test this once with the fix for that
issue, but I think it is important if we can isolate the test due to which
this problem has occurred.
Problem-3
----------------
I am seeing as Assertion failure (in ExitParallelMode()) with this test,
but that seems to be an issue due to the lack of integration with
access-parallel-safety patch. I will test this after integrating with
access-parallel-safety patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 16, 2015 at 9:40 AM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp>
wrote:
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.
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;
}
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.
In anycase, I think we need some handling for such cases.
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.
By the way, you can try reproducing this with the example I posted on
Friday.
Sure.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Mar 13, 2015 at 7:06 PM, Alvaro Herrera <alvherre@2ndquadrant.com>
wrote:
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.
Agreed, will update this in next version of patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Import Notes
Reply to msg id not found: 20150313133602.GG3291@alvh.no-ip.org
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Mar 13, 2015 at 7:00 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Mar 17, 2015 at 7:54 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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().
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
Attachments:
stop-notify-fix-v2.patchbinary/octet-stream; name=stop-notify-fix-v2.patchDownload
From 95579c930ee14e2baa60149e7084e4a97b02bc80 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Tue, 17 Mar 2015 11:09:29 -0400
Subject: [PATCH 5/5] Immediately notify about a dead, never-started worker.
---
src/backend/postmaster/bgworker.c | 25 ++++++++++++++++++++++++-
1 file changed, 24 insertions(+), 1 deletion(-)
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index f80141a..fe94c8d 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -244,14 +244,37 @@ BackgroundWorkerStateChange(void)
rw->rw_terminate = true;
if (rw->rw_pid != 0)
kill(rw->rw_pid, SIGTERM);
+ else
+ {
+ /* Report never-started, now-terminated worker as dead. */
+ ReportBackgroundWorkerPID(rw);
+ }
}
continue;
}
- /* If it's already flagged as do not restart, just release the slot. */
+ /*
+ * If the worker is marked for termination, we don't need to add it
+ * to the registered workers list; we can just free the slot.
+ * However, if bgw_notify_pid is set, the process that registered the
+ * worker may need to know that we've processed the terminate request,
+ * so be sure to signal it.
+ */
if (slot->terminate)
{
+ int notify_pid;
+
+ /*
+ * We need a memory barrier here to make sure that the load of
+ * bgw_notify_pid completes before the store to in_use.
+ */
+ notify_pid = slot->worker.bgw_notify_pid;
+ pg_memory_barrier();
+ slot->pid = 0;
slot->in_use = false;
+ if (notify_pid != 0)
+ kill(notify_pid, SIGUSR1);
+
continue;
}
--
1.7.9.6 (Apple Git-31.1)
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 segmentsThis 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Mar 18, 2015 at 10:45 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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 segmentsThis 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.
Good idea. I think for this particular case, we might want to optimize
the work distribution such each worker gets one independent relation
segment to scan.
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.
Agreed.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Mar 18, 2015 at 9:14 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 16, 2015 at 12:58 PM, Amit Langote <
Langote_Amit_f8@lab.ntt.co.jp> wrote:
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
discussion with Robert), so that it restarts from first queue in such
a case.
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?
Fixed this issue by not going for Wait incase of detached queues.
Apart from these fixes, latest patch contains below changes:
1. Integrated with assess-parallel-safety-v4.patch [1]/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com. To test
with this patch, please remember to comment below line
in this patch, else it will always enter parallel-mode.
+ glob->parallelModeNeeded = glob->parallelModeOK; /* XXX JUST FOR
TESTING */
2. Handle the case where enough workers are not available for
execution of Funnel node. In such a case it will run the plan
with available number of workers and incase no worker is available,
it will just run the local partial seq scan node. I think we can
invent some more advanced solution to handle this problem in
case there is a strong need after the first version went in.
3. Support for pg_stat_statements (it will show the stats for parallel-
statement). To handle this case, we need to share buffer usage
stats from all the workers. Currently the patch does collect
buffer usage stats by default (even though pg_stat_statements is
not enabled) as that is quite cheap and we can make it conditional
if required in future.
So the patches have to be applied in below sequence:
HEAD Commit-id : 8d1f2390
parallel-mode-v8.1.patch [2]/messages/by-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
assess-parallel-safety-v4.patch [1]/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
parallel-heap-scan.patch [3]/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
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]: /messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2]: /messages/by-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
/messages/by-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
[3]: /messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v11.patchapplication/octet-stream; name=parallel_seqscan_v11.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 383e15b..d384e8f 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1582,6 +1582,20 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
}
/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index c639552..8b2cc26 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +918,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1073,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1216,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1322,6 +1350,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1331,6 +1360,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2214,6 +2251,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..10dc319 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -155,6 +157,14 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +468,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index 1c8be25..f13b7bcb 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 5f361d2..5bc4da2 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
@@ -318,6 +320,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..8af19a4
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,408 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ currentRelation,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ estate->toc = node->pcxt->toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ /* Initialize scan state of workers. */
+ funnelstate->all_workers_done = false;
+ funnelstate->local_scan_done = false;
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * If parallel context is set and workers are not registered,
+ * register them now. If there are no more workers available,
+ * then the funnel node will just scan locally, however we will
+ * retry launcing the workers in each pass so that if some new
+ * worker becomes available we can use the same.
+ */
+ if (node->pcxt && !node->fs_workersReady)
+ {
+ bool any_worker_launched = false;
+
+ /* Register backend workers. */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ /*
+ * if required by plugin, aggregate the buffer usage stats
+ * from all workers.
+ */
+ if (TupIsNull(slot))
+ {
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->ss.ps.state->total_time)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ if (node->pcxt && node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ }
+ else if (node->pcxt)
+ {
+ int i;
+
+ /*
+ * We only need to free the memory allocated to initialize
+ * parallel workers as workers are still not started.
+ */
+ dlist_delete(&node->pcxt->node);
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(node->pcxt->worker[i].error_mqh);
+ node->pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /*
+ * If we have allocated a shared memory segment, detach it. This will
+ * implicitly detach the error queues, and any other shared memory
+ * queues, stored there.
+ */
+ if (node->pcxt->seg != NULL)
+ dsm_detach(node->pcxt->seg);
+
+ /* Free the worker array itself. */
+ pfree(node->pcxt->worker);
+ node->pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(node->pcxt);
+ }
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ EState *estate = node->ss.ps.state;
+
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->fs_workersReady)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+
+ estate->toc = node->pcxt->toc;
+
+ ExecReScan(node->ss.ps.lefttree);
+}
+
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..55aa266
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,288 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/execdebug.h"
+#include "executor/nodeSeqscan.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss_currentScanDesc;
+ estate = node->ps.state;
+ direction = estate->es_direction;
+ slot = node->ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ HeapScanDesc currentScanDesc;
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory.
+ */
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ currentScanDesc = heap_beginscan_parallel(currentRelation, pscan);
+
+ node->ss_currentRelation = currentRelation;
+ node->ss_currentScanDesc = currentScanDesc;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(node, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ps.plan = (Plan *) node;
+ scanstate->ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ps);
+ ExecInitScanTupleSlot(estate, scanstate);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ps);
+ ExecAssignScanProjectionInfo(scanstate);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss_currentRelation;
+ scanDesc = node->ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ HeapScanDesc scan;
+ ParallelHeapScanDesc pscan;
+ EState *estate = node->ps.state;
+
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ scan = node->ss_currentScanDesc;
+
+ heap_parallel_rescan(pscan, /* scan desc */
+ scan); /* new scan keys */
+
+ ExecScanReScan((ScanState *) node);
+}
\ No newline at end of file
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..e4933e6
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,280 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 5994433..01e951b 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -355,6 +355,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4053,6 +4090,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1aa1f55..05d4b3c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -440,6 +440,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2898,6 +2916,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..6b633d6 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,186 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..d4570f2 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,92 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1495,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 1a0d358..874c272 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..d152d73
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,83 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "nodes/relation.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "optimizer/restrictinfo.h"
+#include "optimizer/clauses.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..c8422c9 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,87 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3431,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 84560bc..83576c4 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -275,6 +275,51 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..ef8c317 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index acfd0bc..f649639 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2167,6 +2167,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index faca30b..0e5fd3a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..a06c38f
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,425 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "commands/dbcommands.h"
+#include "executor/nodeFunnel.h"
+#include "miscadmin.h"
+#include "nodes/parsenodes.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 3);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+ ParallelHeapScanDesc pscan;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space, char **buffer_usage_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options,
+ &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 9b2e7f3..0c6b481 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7c18298..bc967ee 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1192,6 +1194,96 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins to report the total usage for statement execution.
+ */
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9c74ed3..fc1d639 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2557,6 +2559,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2744,6 +2756,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 110983f..06c5969 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +504,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index d36e738..0a34b48 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -117,6 +117,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..0d28606 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..3af3a0e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..cb05be7
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 59b17f3..f829175 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +391,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1213,6 +1227,41 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScan uses a bare SeqScanState as its state node, since
+ * it needs no additional fields.
+ */
+typedef SeqScanState PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 38469ef..3f3d572 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index fefddb5..b17021f 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,10 +20,15 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
#include "nodes/lockoptions.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
/* Possible sources of a Query */
typedef enum QuerySource
@@ -156,6 +161,17 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 5f0ea1c..7cdf632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -281,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 72eb49b..c3e1f6a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -741,6 +741,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index cd62aec..7bc7d7e 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..bf91824
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,40 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_BUFF_USAGE 2
+#define PARALLEL_KEY_INST_OPTIONS 3
+#define PARALLEL_KEY_INST_INFO 4
+#define PARALLEL_KEY_TUPLE_QUEUE 5
+#define PARALLEL_KEY_SCAN 6
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index b3c705f..5c25627 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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 needsto 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
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] -
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] -
/messages/by-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
[3] -
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
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]/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
assess-parallel-safety-v4.patch [1]/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
parallel-heap-scan.patch [3]/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
parallel_seqscan_v12.patch (Attached with this mail)
[1]: /messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2]: /messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3]: /messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v12.patchapplication/octet-stream; name=parallel_seqscan_v12.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 6370c1f..22b3cc7 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1595,6 +1595,20 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
}
/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 771f6a8..cdf172c 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +918,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1073,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1216,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1322,6 +1350,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1331,6 +1360,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2218,6 +2255,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..10dc319 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -155,6 +157,14 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +468,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index d87be96..657b928 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 143c56d..d4c9119 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
@@ -318,6 +320,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..71d4e3f
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,354 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ bool any_worker_launched = false;
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * If there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ /*
+ * if required by plugin, aggregate the buffer usage stats
+ * from all workers.
+ */
+ if (TupIsNull(slot))
+ {
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->ss.ps.state->total_time)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ }
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->pcxt)
+ {
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+
+ ExecReScan(node->ss.ps.lefttree);
+}
+
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..99cd691
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,319 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We pass 'toc' (place to lookup parallel scan
+ * descriptor) via EState for parallel workers whereas master backend
+ * stores it directly in partial scan state node.
+ */
+ if (estate->toc)
+ node->ss.ps.toc = estate->toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ {
+ /*HeapScanDesc scan;
+ ParallelHeapScanDesc pscan;
+ EState *estate = node->ss.ps.state;
+
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ scan = node->ss.ss_currentScanDesc;
+
+ heap_parallel_rescan(pscan, scan);*/
+
+ node->scan_initialized = false;
+ }
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..e4933e6
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,280 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d8c9a0e..3c0123a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -355,6 +355,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4049,6 +4086,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1aa1f55..05d4b3c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -440,6 +440,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2898,6 +2916,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..6b633d6 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,186 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..d4570f2 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,92 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1495,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 1a0d358..874c272 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..949e79b
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,80 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..c8422c9 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,87 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3431,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1824e7b..4717f78 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -275,6 +275,51 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..ef8c317 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index acfd0bc..f649639 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2167,6 +2167,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index faca30b..0e5fd3a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..925bb7a
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,421 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "executor/nodeFunnel.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 3);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+ ParallelHeapScanDesc pscan;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space, char **buffer_usage_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options,
+ &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 9b2e7f3..0c6b481 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7c18298..92da4f8 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1192,6 +1194,98 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins to report the total usage for statement execution.
+ */
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9c74ed3..fc1d639 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2557,6 +2559,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2744,6 +2756,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 110983f..06c5969 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +504,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index d36e738..0a34b48 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -117,6 +117,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..0d28606 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..3af3a0e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..cb05be7
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ac75f86..cd79588 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +391,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1016,6 +1030,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1216,6 +1235,45 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 38469ef..3f3d572 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 4c63b1a..6a94190 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,10 +20,15 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
#include "nodes/lockoptions.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
/* Possible sources of a Query */
typedef enum QuerySource
@@ -156,6 +161,17 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 5f0ea1c..7cdf632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -281,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 72eb49b..c3e1f6a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -741,6 +741,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..8d6e350 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..bf91824
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,40 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_BUFF_USAGE 2
+#define PARALLEL_KEY_INST_OPTIONS 3
+#define PARALLEL_KEY_INST_INFO 4
+#define PARALLEL_KEY_TUPLE_QUEUE 5
+#define PARALLEL_KEY_SCAN 6
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index b3c705f..5c25627 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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…☺
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<mailto: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<http://www.enterprisedb.com/>
On Wed, Mar 25, 2015 at 4:08 PM, Rajeev rastogi <rajeev.rastogi@huawei.com>
wrote:
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) == NULLI 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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 25 March 2015 at 10:27, Amit Kapila <amit.kapila16@gmail.com> wrote:
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] -
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] -
/messages/by-id/CA+TgmoZJjzYnpXChL3gr7NwRUzkAzPMPVKAtDt5sHvC5Cd7RKw@mail.gmail.com
[3] -
/messages/by-id/CA+TgmoYJETgeAXUsZROnA7BdtWzPtqExPJNTV1GKcaVMgSdhug@mail.gmail.com
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] -
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] -
/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3] -
/messages/by-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, 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.
Here's the query plan:
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.
--
Thom
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*
Logs:
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.
Backtrace:
#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
But the issue seems to produce a different backtrace each time...
2nd backtrace:
#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
3rd backtrace:
#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
4th backtrace:
#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
5th backtrace:
#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
--
Thom
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] -
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] -
/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3] -
/messages/by-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]/messages/by-id/CAA4eK1+NwUJ9ik61yGfZBcN85dQuNEvd38_h1zngCdZrGLGQTQ@mail.gmail.com
and I have mentioned the same while replying on assess-parallel-safety
thread. Can you check after applying the patch in mail [1]/messages/by-id/CAA4eK1+NwUJ9ik61yGfZBcN85dQuNEvd38_h1zngCdZrGLGQTQ@mail.gmail.com
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 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.
[1]: /messages/by-id/CAA4eK1+NwUJ9ik61yGfZBcN85dQuNEvd38_h1zngCdZrGLGQTQ@mail.gmail.com
/messages/by-id/CAA4eK1+NwUJ9ik61yGfZBcN85dQuNEvd38_h1zngCdZrGLGQTQ@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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:
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] -
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
[2] -
/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[3] -
/messages/by-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-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.
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 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, so for
the workers to be above the funnel, not below it. So this is
parallelising, just not in a way that will be a win in this case. :( For
the query I posted (SELECT DISTINCT bid FROM pgbench_partitions), the
parallelised version takes 8 times longer to complete. However, I'm
perhaps premature in what I expect from the feature at this stage.
--
Thom
On Wed, Mar 25, 2015 at 9:53 PM, Thom Brown <thom@linux.com> wrote:
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.
Thanks for verification.
..
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,
Here what you are describing is something like parallel partition
scan which is somewhat related but different feature. This
feature will parallelize the scan for an individual table.
so for the workers to be above the funnel, not below it. So this is
parallelising, just not in a way that will be a win in this case. :( For
the query I
posted (SELECT DISTINCT bid FROM pgbench_partitions), the parallelised
version takes 8 times longer to complete.
I think the primary reason for it not performing as per expectation is
because we have either not the set the right values for cost
parameters or changed the existing cost parameters (cost_seq_page)
which makes planner to select parallel plan even though it is costly.
This is similar to the behaviour when user has intentionally disabled
index scan to test sequence scan and then telling that it is performing
slower.
I think if you want to help in this direction, then what will be more useful
is to see what could be the appropriate values of cost parameters for
parallel scan. We have introduced 3 parameters (cpu_tuple_comm_cost,
parallel_setup_cost, parallel_startup_cost) for costing of parallel plans,
so
with your tests if we can decide what is the appropriate value for each of
these parameters such that it chooses parallel plan only when it is better
than non-parallel plan, then that will be really valuable input.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Mar 25, 2015 at 7:09 PM, Thom Brown <thom@linux.com> wrote:
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
query [1]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_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,3)='pgb' AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring ('pgb',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,3)='pgb' AND substring(pg_catalog.quote_ident (n.nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1) AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) = 1 LIMIT 1000; 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).
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).
Currently I have updated the patch to use second approach which
is to execute/scan locally for Funnel node having subplan.
I understand that it is quite interesting if we can have support for
execution of subplans (un-correlated expression subselects) by
parallel workers, but I feel it is better done as a separate patch.
[1]: 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_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,3)='pgb' AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring ('pgb',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,3)='pgb' AND substring(pg_catalog.quote_ident (n.nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1) AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) = 1 LIMIT 1000;
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_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,3)='pgb' AND (SELECT
pg_catalog.count(*) FROM
pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) ||
'.',1,3) = substring
('pgb',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,3)='pgb' AND
substring(pg_catalog.quote_ident
(n.nspname) || '.',1,3) =
substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1)
AND
(SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE
substring(pg_catalog.quote_ident(nspname) ||
'.',1,3) =
substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) =
1 LIMIT 1000;
Query Plan
--------------------------
QUERY PLAN
--------------------------------------------------------------------------------
--------------------------------------------------------------------------------
----------------------------------------------
Limit (cost=10715.89..10715.92 rows=3 width=85)
-> HashAggregate (cost=10715.89..10715.92 rows=3 width=85)
Group Key: (quote_ident((c.relname)::text))
-> Append (cost=8.15..10715.88 rows=3 width=85)
-> Funnel on pg_class c (cost=8.15..9610.67 rows=1
width=64)
Filter: ((relnamespace <> $4) AND (relkind = ANY
('{r,S,v,m
,f}'::"char"[])) AND ("substring"(quote_ident((relname)::text), 1, 3) =
'pgb'::t
ext) AND pg_table_is_visible(oid))
Number of Workers: 1
InitPlan 3 (returns $4)
-> Index Scan using pg_namespace_nspname_index on
pg_nam
espace pg_namespace_2 (cost=0.13..8.15 rows=1 width=4)
Index Cond: (nspname = 'pg_catalog'::name)
-> Partial Seq Scan on pg_class c
(cost=0.00..19043.43 ro
ws=1 width=64)
Filter: ((relnamespace <> $4) AND (relkind = ANY
('{r
,S,v,m,f}'::"char"[])) AND ("substring"(quote_ident((relname)::text), 1, 3)
= 'p
gb'::text) AND pg_table_is_visible(oid))
-> Result (cost=8.52..16.69 rows=1 width=64)
One-Time Filter: ($3 > 1)
InitPlan 2 (returns $3)
-> Aggregate (cost=8.37..8.38 rows=1 width=0)
-> Index Only Scan using
pg_namespace_nspname_inde
x on pg_namespace pg_namespace_1 (cost=0.13..8.37 rows=1 width=0)
Filter:
("substring"((quote_ident((nspname)::
text) || '.'::text), 1, 3) = "substring"('pgb'::text, 1,
(length(quote_ident((ns
pname)::text)) + 1)))
-> Index Only Scan using pg_namespace_nspname_index
on pg_
namespace n (cost=0.13..8.30 rows=1 width=64)
Filter:
("substring"((quote_ident((nspname)::text) ||
'.'::text), 1, 3) = 'pgb'::text)
-> Result (cost=8.79..1088.49 rows=1 width=128)
One-Time Filter: ($0 = 1)
InitPlan 1 (returns $0)
-> Aggregate (cost=8.37..8.38 rows=1 width=0)
-> Index Only Scan using
pg_namespace_nspname_inde
x on pg_namespace (cost=0.13..8.37 rows=1 width=0)
Filter:
("substring"((quote_ident((nspname)::
text) || '.'::text), 1, 3) = "substring"('pgb'::text, 1,
(length(quote_ident((ns
pname)::text)) + 1)))
-> Nested Loop (cost=0.41..1080.09 rows=1 width=128)
-> Index Scan using pg_namespace_oid_index on
pg_nam
espace n_1 (cost=0.13..12.37 rows=1 width=68)
Filter:
("substring"((quote_ident((nspname)::te
xt) || '.'::text), 1, 3) = "substring"('pgb'::text, 1,
(length(quote_ident((nspn
ame)::text)) + 1)))
-> Index Scan using pg_class_relname_nsp_index
on pg
_class c_1 (cost=0.28..1067.71 rows=1 width=68)
Index Cond: (relnamespace = n_1.oid)
Filter: ((relkind = ANY
('{r,S,v,m,f}'::"char"[
])) AND ("substring"(((quote_ident((n_1.nspname)::text) || '.'::text) ||
quote_i
dent((relname)::text)), 1, 3) = 'pgb'::text))
(32 rows)
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v13.patchapplication/octet-stream; name=parallel_seqscan_v13.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 6370c1f..22b3cc7 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -1595,6 +1595,20 @@ heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
}
/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 771f6a8..cdf172c 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +918,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1073,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1216,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1322,6 +1350,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1331,6 +1360,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2218,6 +2255,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..10dc319 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -155,6 +157,14 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +468,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index d87be96..657b928 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 143c56d..d4c9119 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -181,6 +181,8 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
estate->es_param_exec_vals = (ParamExecData *)
palloc0(queryDesc->plannedstmt->nParamExec * sizeof(ParamExecData));
+ estate->toc = queryDesc->toc;
+
/*
* If non-read-only query, set the command ID to mark output tuples with
*/
@@ -318,6 +320,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 022041b..79eeaee 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -145,6 +145,8 @@ CreateExecutorState(void)
estate->es_auxmodifytables = NIL;
+ estate->toc = NULL;
+
estate->es_per_tuple_exprcontext = NULL;
estate->es_epqTuple = NULL;
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..a9e8524c
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,366 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodeFunnel.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ bool any_worker_launched = false;
+
+ /* Initialize the workers required to perform parallel scan. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * For Funnel node to support execution of subplans by parallel
+ * workers, it need to push down the list of subplans stored in
+ * the node and corresponding list of subplans stored in planned
+ * statement as nodes' subplans store reference to subplan in
+ * planned statement. Currently we don't have enough infrastructre
+ * to support executing all kind of nodes by parallel workers, so
+ * it's better to execute such a plan in local node.
+ */
+ if (!node->ss.ps.plan->initPlan)
+ {
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * If there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ /*
+ * if required by plugin, aggregate the buffer usage stats
+ * from all workers.
+ */
+ if (TupIsNull(slot))
+ {
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->ss.ps.state->total_time)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ }
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->pcxt)
+ {
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+
+ ExecReScan(node->ss.ps.lefttree);
+}
+
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..99cd691
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,319 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We pass 'toc' (place to lookup parallel scan
+ * descriptor) via EState for parallel workers whereas master backend
+ * stores it directly in partial scan state node.
+ */
+ if (estate->toc)
+ node->ss.ps.toc = estate->toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ {
+ /*HeapScanDesc scan;
+ ParallelHeapScanDesc pscan;
+ EState *estate = node->ss.ps.state;
+
+ Assert(estate->toc);
+
+ pscan = shm_toc_lookup(estate->toc, PARALLEL_KEY_SCAN);
+
+ scan = node->ss.ss_currentScanDesc;
+
+ heap_parallel_rescan(pscan, scan);*/
+
+ node->scan_initialized = false;
+ }
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..e4933e6
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,280 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static void
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d8c9a0e..3c0123a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -355,6 +355,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4049,6 +4086,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1aa1f55..05d4b3c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -440,6 +440,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2898,6 +2916,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..6b633d6 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, see above */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,186 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..d4570f2 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,92 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1495,12 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 1a0d358..874c272 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..949e79b
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,80 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..c8422c9 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,11 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path,
+ List *tlist, List *scan_clauses);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +105,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +239,8 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -343,6 +356,20 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +573,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1162,87 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3431,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1824e7b..4717f78 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -275,6 +275,51 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = 0;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ec828cd..ef8c317 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,24 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index acfd0bc..f649639 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2167,6 +2167,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index faca30b..0e5fd3a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..925bb7a
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,421 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * InitializeParallelWorkers Setup dynamic shared memory and parallel backend workers.
+ */
+#include "postgres.h"
+
+#include "executor/nodeFunnel.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+static void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size);
+static void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters and instrumentation information that need to be
+ * retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, Size *params_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 3);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters and instrumentation information
+ * required for parallel execution.
+ */
+void
+StoreParallelSupportInfo(ParallelContext *pcxt, ParamListInfo params,
+ int instOptions, int params_size,
+ char **inst_options_space,
+ char **buffer_usage_space)
+{
+ char *paramsdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+ ParallelHeapScanDesc pscan;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, EState *estate, Relation rel,
+ char **inst_options_space, char **buffer_usage_space,
+ shm_mq_handle ***responseqp, ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ estate->es_instrument, ¶ms_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ estate->es_instrument,
+ params_size, inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameter's and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+}
+
+/*
+ * GetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+GetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->qual);
+ fix_opfuncids((Node*) (*plannedstmt)->planTree->targetlist);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ GetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &inst_options,
+ &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->toc = toc;
+ parallelstmt->responseq = responseq;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 9b2e7f3..0c6b481 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..7a9ce3e 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -129,6 +130,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +166,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +209,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +254,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7c18298..92da4f8 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1192,6 +1194,98 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ queryDesc->toc = parallelstmt->toc;
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins to report the total usage for statement execution.
+ */
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..0bbc67b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -80,6 +80,7 @@ CreateQueryDesc(PlannedStmt *plannedstmt,
qd->params = params; /* parameter values passed into query */
qd->instrument_options = instrument_options; /* instrumentation
* wanted? */
+ qd->toc = NULL; /* need to be set by the caller before ExecutorStart */
/* null these fields until set by ExecutorStart */
qd->tupDesc = NULL;
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9c74ed3..fc1d639 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -608,6 +608,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2557,6 +2559,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2744,6 +2756,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 110983f..06c5969 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +504,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index d36e738..0a34b48 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -117,6 +117,7 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
diff --git a/src/include/executor/execdesc.h b/src/include/executor/execdesc.h
index a2381cd..56b7c75 100644
--- a/src/include/executor/execdesc.h
+++ b/src/include/executor/execdesc.h
@@ -42,6 +42,7 @@ typedef struct QueryDesc
DestReceiver *dest; /* the destination for tuple output */
ParamListInfo params; /* param values being passed in */
int instrument_options; /* OR of InstrumentOption flags */
+ shm_toc *toc; /* to fetch the information from dsm */
/* These fields are set by ExecutorStart */
TupleDesc tupDesc; /* descriptor for result tuples */
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index 1c3b2b0..0d28606 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..3af3a0e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..cb05be7
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..c979233
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,34 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ac75f86..cd79588 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +391,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1016,6 +1030,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1216,6 +1235,45 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 38469ef..3f3d572 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..65b60a0 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -103,4 +103,9 @@ typedef struct ParamExecData
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 4c63b1a..6a94190 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,10 +20,15 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
#include "nodes/lockoptions.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/shm_toc.h"
+#include "storage/shm_mq.h"
/* Possible sources of a Query */
typedef enum QuerySource
@@ -156,6 +161,17 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ shm_toc *toc;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 5f0ea1c..7cdf632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -281,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 72eb49b..c3e1f6a 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -741,6 +741,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..8d6e350 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..bf91824
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,40 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_BUFF_USAGE 2
+#define PARALLEL_KEY_INST_OPTIONS 3
+#define PARALLEL_KEY_INST_INFO 4
+#define PARALLEL_KEY_TUPLE_QUEUE 5
+#define PARALLEL_KEY_SCAN 6
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan, EState *estate,
+ Relation rel, char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..b560672 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index b3c705f..5c25627 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 30, 2015 at 8:11 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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]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). 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.
[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 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).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Mar 30, 2015 at 8:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Mar 30, 2015 at 8:31 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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?
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?
+ if (!parallel)
+ 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);
+ }
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Apr 1, 2015 at 6:03 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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)?
[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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Apr 1, 2015 at 8:18 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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]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_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,3)='pgb' AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring ('pgb',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,3)='pgb' AND substring(pg_catalog.quote_ident (n.nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1) AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) = 1 LIMIT 1000;. In this, the initPlan is generated for Qualification expression
[2]: (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)
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.
So now the point I am not able to understand from your explanation
is that how the worker will perform qualification without the knowledge
of initPlan?
[1]: 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_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,3)='pgb' AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring ('pgb',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,3)='pgb' AND substring(pg_catalog.quote_ident (n.nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1) AND (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) = 1 LIMIT 1000;
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_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,3)='pgb' AND (SELECT
pg_catalog.count(*) FROM
pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) ||
'.',1,3) = substring
('pgb',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,3)='pgb' AND
substring(pg_catalog.quote_ident
(n.nspname) || '.',1,3) =
substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(n.nspname))+1)
AND
(SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE
substring(pg_catalog.quote_ident(nspname) ||
'.',1,3) =
substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)) =
1 LIMIT 1000;
[2]: (SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE substring(pg_catalog.quote_ident(nspname) || '.',1,3) = substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)
(SELECT pg_catalog.count(*) FROM pg_catalog.pg_namespace WHERE
substring(pg_catalog.quote_ident(nspname) ||
'.',1,3) =
substring('pgb',1,pg_catalog.length(pg_catalog.quote_ident(nspname))+1)
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 Cpostgres.exe!ExecEvalParamExec(ExprState * exprstate=0x000000000c33f980,
ExprContext * econtext=0x000000000c33de50, char * isNull=0x000000000c33f481,
ExprDoneCond * isDone=0x0000000000000000) Line 1111 Cpostgres.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 CBasically 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
So I've just finished reading the impressive 244 emails (so far) about
Parallel Seq scan, and I've had a quick skim over the latest patch.
Its quite exciting to think that one day we'll have parallel query in
PostgreSQL, but I have to say, that I think that there's a major point
about the proposed implementation that seems to have gotten forgotten
about, which I can't help but think won't get that far off the ground
unless more thought goes into it.
On 11 February 2015 at 09:56, Andres Freund <andres@2ndquadrant.com> wrote:
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.
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.
Instead what I think should happen is:
The following link has been seen before on this thread, but I'll post it
again:
http://docs.oracle.com/cd/A57673_01/DOC/server/doc/A48506/pqoconce.htm
There's one key sentence in there that should not be ignored:
"It is important to note that the query is parallelized dynamically at
execution time."
"dynamically at execution time"... I think this also needs to happen in
PostgreSQL. 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 (Likely this is not too important
to nail this down for a first implementation).
* The plan should then be completely traversed node by node to determine
which nodes can be made parallel. This would likely require an interface
function to each node which returns true or false, depending on if it's
safe to parallelise. For seq scan this could be a simple test to see if
we're scanning a temp table.
* Before any changes are made to the plan, a complete copy of it should be
made.
* Funnel nodes could then be injected below the last node in each branch
which supports parallelism. If more than one branch exists with parallel
enabled nodes, then it should be up to this function to determine, based on
cost, which nodes will benefit the most from the additional workers.
Certain other node types would need something else below the Funnel node,
e.g Partial aggregation would need a new node below the Funnel to complete
the aggregation.
* The first parallel enabled nodes should be passed off to the worker
processes for execution.
So I quite strongly agree with Andres' comment above that we really need to
move away from this 1:1 assumption about the relationship between plan
nodes and executor nodes. Tom did mention some possible reasons here in his
response to my INNER JOIN removals patch ->
/messages/by-id/32139.1427667410@sss.pgh.pa.us
Tom wrote:
"What you're doing here violates the rule that planstate trees have a
one-to-one relationship to plan trees. EXPLAIN used to iterate over those
trees in lockstep, and there probably still is code that does similar
things (in third-party modules if not core), so I don't think we should
abandon that principle."
So perhaps this needs analysis. If it's not possible, then perhaps the
parallel nodes could be inserted at the end of planning, providing the
executor could be coded in such a way that the parallel plan can still work
with 0 worker processes. Unfortunately it seems that transitions through
nodes that don't do anything is not free, so with this method there would
be a slowdown of parallel enabled plans when they're executed without any
worker processes.
Also here ->
https://technet.microsoft.com/en-us/library/ms178065%28v=sql.105%29.aspx
There's some text that says:
"The SQL Server query optimizer does not use a parallel execution plan for
a query if any one of the following conditions is true:"
"* A serial execution plan is considered faster than any possible parallel
execution plan for the particular query."
I'm finding it a bit hard to get a true meaning from that, but if I'm not
mistaken it means that the serial plan will be preferred over a parallel
plan, as if the parallel plan does not get allocated any workers at
execution time, then we don't want to be left with a slow plan...
Apologies if any of this has been discussed any already designed around, I
just didn't see anything in the emails to indicate that it has.
Regards
David Rowley
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 parallelisedThere 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 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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Apr 8, 2015 at 7:54 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
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.
But with this, and the parallel costing model above, to know the cost of a
parallel path, you need to know how many workers will be available later at
execution time in order to know what that percentage is, or would we just
always assume we'd get max_parallel_degree each time the plan is executed,
similar to how the latest patch works?
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.
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?
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.
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.
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
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.
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.
Regards
David Rowley
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 8 April 2015 at 15:46, Amit Kapila <amit.kapila16@gmail.com> wrote:
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 makeparallel */
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 parallelisedThere 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 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.
If the planner didn't know about parallelism then we'd already have the
most optimal plan and it would be no great loss if no workers were around
to help.
Regards
David Rowley
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 resources
are not available during execution, but same is true for work_mem
as well for a non-parallel plan. I think we need some advanced way
to handle the case when resources are not available during execution
by either re-planing the statement or by some other way, but that can
also be done separately.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 9 April 2015 at 00:12, Amit Kapila <amit.kapila16@gmail.com> wrote:
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-startphase
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 (particularlyparallel-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.
Yes this is true, but if we already have the most optimal serial plan, then
there's no issue.
I agree that there is a chance of loss incase appropriate resources
are not available during execution, but same is true for work_mem
as well for a non-parallel plan. I think we need some advanced way
to handle the case when resources are not available during execution
by either re-planing the statement or by some other way, but that can
also be done separately.
There was some talk of re-planning queries over on the Removing INNER JOINs
thread:
/messages/by-id/CA+TgmoaHi8tq7haZCf46O_NUHT8w=P0Z_N59DC0yOjfMucS9bg@mail.gmail.com
Regards
David Rowley
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 21 April 2015 at 06:26, Robert Haas <robertmhaas@gmail.com> wrote:
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 overheadThe third one isn't really true. You've just moved some of the
planning to execution time.
Hmm, sorry, I meant no planner overhead during normal planning.
I was more driving along the lines of the fact that low cost queries don't
have to pay the price for the planner considering parallel paths. This
"parallelizer" that I keep talking about would only be asked to do anything
if the root node's cost was above some GUC like parallel_cost_threshold,
and likely a default for this would be some cost that would translate into
a query that took, say roughly anything over 1 second. This way super fast
1 millisecond plans don't have to suffer from extra time taken to consider
parallel paths. Once we're processing queries that are above this parallel
threshold then the cost of the parallelizer invocation would be drowned out
by the actual execution cost anyway.
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 bythe
time you inject Funnel nodes.
If we parallelise during planning time:
Pros
* More chance of getting a parallel friendly plan which could end upbeing
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.
I agree with that. It's a tough one.
I was hoping that this might be offset by the fact that we won't have to
pay the high price when the planner spits out a parallel plan when the
executor has no spare workers to execute it as intended, and also the we
wouldn't have to be nearly as conservative with the max_parallel_degree
GUC, that could just be set to the number of logical CPUs in the machine,
and we could just use that value minus number of active backends during
execution.
Cons:
* May produce non optimal plans if no worker processes are availableduring
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.
Apologies if this seems like complete rubbish, or if it seems like parallel
query mark 3, when we're not done yet with mark 1. I just can't see how,
with the current approach how we could just parallelise normal plans like
the 4 way join I describe above and I think it would be a shame if we
developed down a path that made this not possible.
Regards
David Rowley
On Tue, Apr 21, 2015 at 2:29 PM, David Rowley <dgrowleyml@gmail.com> wrote:
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?
The Funnel node is quite generic and can handle the case as
described by you if we add Funnel on top of join node (c join d).
It currently passes plannedstmt to worker which can contain any
type of plan (though we need to add some more code to make it
work if want to execute any node other than Result or PartialSeqScan
node.)
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.
I think there is some advantage of changing an already built plan
to parallel plan based on resources and there is some literature
about the same, but I think we will loose much more by not considering
parallelism during planning time. If I remember correctly, then some
of the other databases do tackle this problem of shortage of resources
during execution as mentioned by me upthread, but I think for that it
is not necessary to have a Parallel Planner as a separate layer.
I believe it is important to have some way to handle shortage of resources
during execution, but it can be done at later stage.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Apr 21, 2015 at 6:34 AM, Amit Langote <Langote_Amit_f8@lab.ntt.co.jp
wrote:
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"
What I am planning to do is Destroy the resources (parallel context) once
we have fetched all the tuples from Funnel node, so that we don't block
all resources till end of execution. We can't say that as reuse rather it
will allow multiple nodes in same statement to use workers when there
is a restriction on total number of workers (max_worker_processed) that
can be used.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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?
If yes, then I am able to understand the advantage of retaining both
parallel and non-parallel paths, else could you explain some more
why you think it is advantageous to retain parallel-path even when it
losses to serial path in the beginning?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Mar 30, 2015 at 8:31 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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]/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
assess-parallel-safety-v4.patch [2]/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com (don't forget to run fixpgproc.pl in
the patch)
parallel_seqscan_v14.patch (Attached with this mail)
[1]: /messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
/messages/by-id/CA+TgmoZfSXZhS6qy4Z0786D7iU_AbhBVPQFwLthpSvGieczqHg@mail.gmail.com
[2]: /messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
/messages/by-id/CA+TgmobJSuefiPOk6+i9WERUgeAB3ggJv7JxLX+r6S5SYydBRQ@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v14.patchapplication/octet-stream; name=parallel_seqscan_v14.patchDownload
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..639451a 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -26,9 +26,9 @@
static void printtup_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-static void printtup(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_20(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
static void printtup_shutdown(DestReceiver *self);
static void printtup_destroy(DestReceiver *self);
@@ -299,7 +299,7 @@ printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
* printtup --- print a tuple in protocol 3.0
* ----------------
*/
-static void
+static bool
printtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -376,13 +376,15 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
* printtup_20 --- print a tuple in protocol 2.0
* ----------------
*/
-static void
+static bool
printtup_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -452,6 +454,8 @@ printtup_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
@@ -528,7 +532,7 @@ debugStartup(DestReceiver *self, int operation, TupleDesc typeinfo)
* debugtup - print one tuple for an interactive backend
* ----------------
*/
-void
+bool
debugtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -553,6 +557,8 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
printatt((unsigned) i + 1, typeinfo->attrs[i], value);
}
printf("\t----\n");
+
+ return true;
}
/* ----------------
@@ -564,7 +570,7 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
* This is largely same as printtup_20, except we use binary formatting.
* ----------------
*/
-static void
+static bool
printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -636,4 +642,6 @@ printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index da0b70e..388a8c6 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -63,6 +63,7 @@
#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
+#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/datum.h"
#include "utils/inval.h"
@@ -80,8 +81,10 @@ bool synchronize_seqscans = true;
static HeapScanDesc heap_beginscan_internal(Relation relation,
Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync,
bool is_bitmapscan, bool temp_snap);
+static BlockNumber heap_parallelscan_nextpage(ParallelHeapScanDesc);
static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
TransactionId xid, CommandId cid, int options);
static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
@@ -222,7 +225,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool is_rescan)
* results for a non-MVCC snapshot, the caller must hold some higher-level
* lock that ensures the interesting tuple(s) won't change.)
*/
- scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
+ if (scan->rs_parallel != NULL)
+ scan->rs_nblocks = scan->rs_parallel->phs_nblocks;
+ else
+ scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
/*
* If the table is large relative to NBuffers, use a bulk-read access
@@ -481,7 +487,18 @@ heapgettup(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineoff = FirstOffsetNumber; /* first offnum */
scan->rs_inited = true;
@@ -504,6 +521,9 @@ heapgettup(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -656,11 +676,19 @@ heapgettup(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -758,7 +786,18 @@ heapgettup_pagemode(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineindex = 0;
scan->rs_inited = true;
@@ -778,6 +817,9 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -919,11 +961,19 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -1316,7 +1366,7 @@ HeapScanDesc
heap_beginscan(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, false, false);
}
@@ -1326,7 +1376,7 @@ heap_beginscan_catalog(Relation relation, int nkeys, ScanKey key)
Oid relid = RelationGetRelid(relation);
Snapshot snapshot = RegisterSnapshot(GetCatalogSnapshot(relid));
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, false, true);
}
@@ -1335,7 +1385,7 @@ heap_beginscan_strat(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, false, false);
}
@@ -1343,13 +1393,14 @@ HeapScanDesc
heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
false, false, true, false);
}
static HeapScanDesc
heap_beginscan_internal(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync,
bool is_bitmapscan, bool temp_snap)
{
@@ -1377,6 +1428,7 @@ heap_beginscan_internal(Relation relation, Snapshot snapshot,
scan->rs_allow_strat = allow_strat;
scan->rs_allow_sync = allow_sync;
scan->rs_temp_snap = temp_snap;
+ scan->rs_parallel = parallel_scan;
/*
* we can use page-at-a-time mode if it's an MVCC-safe snapshot
@@ -1470,6 +1522,93 @@ heap_endscan(HeapScanDesc scan)
}
/* ----------------
+ * heap_parallelscan_estimate - estimate storage for ParallelHeapScanDesc
+ *
+ * Sadly, this doesn't reduce to a constant, because the size required
+ * to serialize the snapshot can vary.
+ * ----------------
+ */
+Size
+heap_parallelscan_estimate(Snapshot snapshot)
+{
+ return add_size(offsetof(ParallelHeapScanDescData, phs_snapshot_data),
+ EstimateSnapshotSpace(snapshot));
+}
+
+/* ----------------
+ * heap_parallelscan_initialize - initialize ParallelHeapScanDesc
+ *
+ * Must allow as many bytes of shared memory as returned by
+ * heap_parallelscan_estimate. Call this just once in the leader
+ * process; then, individual workers attach via heap_beginscan_parallel.
+ * ----------------
+ */
+void
+heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
+ Snapshot snapshot)
+{
+ target->phs_relid = RelationGetRelid(relation);
+ target->phs_nblocks = RelationGetNumberOfBlocks(relation);
+ SpinLockInit(&target->phs_mutex);
+ target->phs_cblock = 0;
+ SerializeSnapshot(snapshot, target->phs_snapshot_data);
+}
+/* ----------------
+ * heap_parallelscan_nextpage - get the next page to scan
+ *
+ * A return value larger than the number of blocks to be scanned
+ * indicates end of scan. Note, however, that other backends could still
+ * be scanning if they grabbed a page to scan and aren't done with it yet.
+ * ----------------
+ */
+static BlockNumber
+heap_parallelscan_nextpage(ParallelHeapScanDesc parallel_scan)
+{
+ BlockNumber page = InvalidBlockNumber;
+
+ /* we treat InvalidBlockNumber specially here to avoid overflow */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock != InvalidBlockNumber)
+ page = parallel_scan->phs_cblock++;
+ SpinLockRelease(¶llel_scan->phs_mutex);
+
+ return page;
+}
+
+/* ----------------
+ * heap_beginscan_parallel - join a parallel scan
+ *
+ * Caller must hold a suitable lock on the correct relation.
+ * ----------------
+ */
+HeapScanDesc
+heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
+{
+ Snapshot snapshot;
+
+ Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
+ snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
+ RegisterSnapshot(snapshot);
+
+ return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
+ true, true, false, true);
+}
+
+/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 0d3721a..612c469 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -4393,7 +4393,7 @@ copy_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* copy_dest_receive --- receive one tuple
*/
-static void
+static bool
copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_copy *myState = (DR_copy *) self;
@@ -4405,6 +4405,8 @@ copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
/* And send the data */
CopyOneRowTo(cstate, InvalidOid, slot->tts_values, slot->tts_isnull);
myState->processed++;
+
+ return true;
}
/*
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index 54b2f38..68db546 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -62,7 +62,7 @@ typedef struct
static ObjectAddress CreateAsReladdr = {InvalidOid, InvalidOid, 0};
static void intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void intorel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool intorel_receive(TupleTableSlot *slot, DestReceiver *self);
static void intorel_shutdown(DestReceiver *self);
static void intorel_destroy(DestReceiver *self);
@@ -482,7 +482,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* intorel_receive --- receive one tuple
*/
-static void
+static bool
intorel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_intorel *myState = (DR_intorel *) self;
@@ -507,6 +507,8 @@ intorel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 771f6a8..cdf172c 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -721,6 +721,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -916,6 +918,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1065,6 +1073,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1206,6 +1216,24 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for parallel sequence scan.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ {
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers = ((FunnelState *)planstate)->pcxt->nworkers;
+ char *inst_info_workers = ((FunnelState *)planstate)->inst_options_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(planstate->instrument, instrument_worker);
+ }
+ }
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1322,6 +1350,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1331,6 +1360,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2218,6 +2255,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index eb16bb3..78f822b 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -56,7 +56,7 @@ typedef struct
static int matview_maintenance_depth = 0;
static void transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
static void transientrel_shutdown(DestReceiver *self);
static void transientrel_destroy(DestReceiver *self);
static void refresh_matview_datafill(DestReceiver *dest, Query *query,
@@ -422,7 +422,7 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* transientrel_receive --- receive one tuple
*/
-static void
+static bool
transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_transientrel *myState = (DR_transientrel *) self;
@@ -441,6 +441,8 @@ transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index af707b0..991ff51 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -16,14 +16,15 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execJunk.o execMain.o \
execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
- nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
+ nodeSeqscan.o nodePartialSeqscan.o nodeSetOp.o nodeSort.o \
+ nodeUnique.o nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o \
+ spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 6ebad2f..10dc319 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -155,6 +157,14 @@ ExecReScan(PlanState *node)
ExecReScanSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -458,6 +468,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index d87be96..657b928 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -261,6 +261,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
* Relation scan nodes can all be treated alike
*/
case T_SeqScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index dc7d506..9338591 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -318,6 +318,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
@@ -1550,7 +1553,15 @@ ExecutePlan(EState *estate,
* practice, this is probably always the case at this point.)
*/
if (sendTuples)
- (*dest->receiveSlot) (slot, dest);
+ {
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
+ }
/*
* Count tuples processed, if this is a SELECT. (For other operation
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 9892499..1a1275c 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSeqscan.h"
@@ -190,6 +192,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -406,6 +418,14 @@ ExecProcNode(PlanState *node)
result = ExecSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -644,6 +664,14 @@ ExecEndNode(PlanState *node)
ExecEndSeqScan((SeqScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index 753754d..c874a27 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -1266,7 +1266,7 @@ do_tup_output(TupOutputState *tstate, Datum *values, bool *isnull)
ExecStoreVirtualTuple(slot);
/* send the tuple to the receiver */
- (*tstate->dest->receiveSlot) (slot, tstate->dest);
+ (void) (*tstate->dest->receiveSlot) (slot, tstate->dest);
/* clean up */
ExecClearTuple(slot);
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 0736d2a..fdb2c82 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -1512,3 +1512,28 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
MemoryContextSwitchTo(oldcontext);
}
+
+/*
+ * Populate the values of PARAM_EXEC parameters.
+ *
+ * This is used by worker backends to fill in the values
+ * of PARAM_EXEC parameters after fetching the same from
+ * dynamic shared memory. This needs to be called before
+ * ExecutorRun.
+ */
+void
+PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals)
+{
+ ListCell *lparam;
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].value =
+ param_val->value;
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].isnull =
+ param_val->isnull;
+ }
+}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 812a610..863bd64 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -167,7 +167,7 @@ static Datum postquel_get_single_result(TupleTableSlot *slot,
static void sql_exec_error_callback(void *arg);
static void ShutdownSQLFunction(Datum arg);
static void sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
static void sqlfunction_shutdown(DestReceiver *self);
static void sqlfunction_destroy(DestReceiver *self);
@@ -1903,7 +1903,7 @@ sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* sqlfunction_receive --- receive one tuple
*/
-static void
+static bool
sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_sqlfunction *myState = (DR_sqlfunction *) self;
@@ -1913,6 +1913,8 @@ sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
/* Store the filtered tuple into the tuplestore */
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..a08e24c
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,394 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodeSubplan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ ExprContext *econtext = node->ss.ps.ps_ExprContext;
+ bool any_worker_launched = false;
+ List *serialized_param_exec;
+
+ /*
+ * 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);
+
+ /* Initialize the workers required to execute funnel node. */
+ InitializeParallelWorkers(node->ss.ps.plan->lefttree,
+ serialized_param_exec,
+ estate,
+ node->ss.ss_currentRelation,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * If there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ /*
+ * if required by plugin, aggregate the buffer usage stats
+ * from all workers.
+ */
+ if (TupIsNull(slot))
+ {
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->ss.ps.state->total_time)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+
+ /*
+ * Destroy the parallel context once we complete fetching all
+ * the tuples, this will ensure that if in the same statement
+ * we need to have Funnel node for multiple parts of statement,
+ * it won't accumulate lot of dsm's and workers can be made
+ * available to use by other parts of statement.
+ */
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ WaitForParallelWorkersToFinish(node->pcxt);
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+ node->funnel = NULL;
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * Re-initialize the parallel context and workers to perform
+ * rescan of relation.
+ */
+ if (node->pcxt)
+ {
+ /*
+ * 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.
+ */
+ if (node->fs_workersReady)
+ {
+ TupleQueueFunnelShutdown(node->funnel);
+ WaitForParallelWorkersToFinish(node->pcxt);
+ }
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+ node->funnel = NULL;
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+
+ ExecReScan(node->ss.ps.lefttree);
+}
+
diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c
index e66bcda..c447062 100644
--- a/src/backend/executor/nodeNestloop.c
+++ b/src/backend/executor/nodeNestloop.c
@@ -144,6 +144,7 @@ ExecNestLoop(NestLoopState *node)
{
NestLoopParam *nlp = (NestLoopParam *) lfirst(lc);
int paramno = nlp->paramno;
+ TupleDesc tdesc = outerTupleSlot->tts_tupleDescriptor;
ParamExecData *prm;
prm = &(econtext->ecxt_param_exec_vals[paramno]);
@@ -154,6 +155,7 @@ ExecNestLoop(NestLoopState *node)
prm->value = slot_getattr(outerTupleSlot,
nlp->paramval->varattno,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[nlp->paramval->varattno-1]->atttypid;
/* Flag parameter value as changed */
innerPlan->chgParam = bms_add_member(innerPlan->chgParam,
paramno);
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..730ee2e
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,308 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/nodePartialSeqscan.h"
+#include "postmaster/backendworker.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ shm_toc *toc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We set 'toc' (place to lookup parallel scan
+ * descriptor) as retrievied by attaching to dsm for parallel workers
+ * whereas master backend stores it directly in partial scan state node
+ * after initializing workers.
+ */
+ toc = GetParallelShmToc();
+ if (toc)
+ node->ss.ps.toc = toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ node->scan_initialized = false;
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c
index 8d3dde0..b348bfd 100644
--- a/src/backend/executor/nodeResult.c
+++ b/src/backend/executor/nodeResult.c
@@ -75,6 +75,13 @@ ExecResult(ResultState *node)
econtext = node->ps.ps_ExprContext;
/*
+ * Result node can be added as a gating node on top of PartialSeqScan
+ * node, so need to percolate toc information to outer node.
+ */
+ if (node->ps.toc)
+ outerPlanState(node)->toc = node->ps.toc;
+
+ /*
* check constant qualifications like (2 > 1), if not already done
*/
if (node->rs_checkqual)
diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c
index 9eb4d63..6afd55a 100644
--- a/src/backend/executor/nodeSubplan.c
+++ b/src/backend/executor/nodeSubplan.c
@@ -30,11 +30,14 @@
#include <math.h>
#include "access/htup_details.h"
+#include "catalog/pg_type.h"
#include "executor/executor.h"
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "utils/array.h"
+#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
@@ -281,12 +284,14 @@ ExecScanSubPlan(SubPlanState *node,
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -399,6 +404,7 @@ ExecScanSubPlan(SubPlanState *node,
prmdata = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col, &(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
@@ -551,6 +557,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
!TupIsNull(slot);
slot = ExecProcNode(planstate))
{
+ TupleDesc tdesc = slot->tts_tupleDescriptor;
int col = 1;
ListCell *plst;
bool isnew;
@@ -568,6 +575,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col,
&(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
slot = ExecProject(node->projRight, NULL);
@@ -954,6 +962,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
ListCell *l;
bool found = false;
ArrayBuildStateAny *astate = NULL;
+ Oid ptype;
if (subLinkType == ANY_SUBLINK ||
subLinkType == ALL_SUBLINK)
@@ -961,6 +970,8 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (subLinkType == CTE_SUBLINK)
elog(ERROR, "CTE subplans should not be executed via ExecSetParamPlan");
+ ptype = exprType((Node *) node->xprstate.expr);
+
/* Initialize ArrayBuildStateAny in caller's context, if needed */
if (subLinkType == ARRAY_SUBLINK)
astate = initArrayResultAny(subplan->firstColType,
@@ -983,12 +994,14 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -1011,6 +1024,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(true);
+ prm->ptype = ptype;
prm->isnull = false;
found = true;
break;
@@ -1062,6 +1076,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = heap_getattr(node->curTuple, i, tdesc,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[i-1]->atttypid;
i++;
}
}
@@ -1084,6 +1099,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
true);
prm->execPlan = NULL;
prm->value = node->curArray;
+ prm->ptype = ptype;
prm->isnull = false;
}
else if (!found)
@@ -1096,6 +1112,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(false);
+ prm->ptype = ptype;
prm->isnull = false;
}
else
@@ -1108,6 +1125,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = (Datum) 0;
+ prm->ptype = VOIDOID;
prm->isnull = true;
}
}
@@ -1238,3 +1256,47 @@ ExecAlternativeSubPlan(AlternativeSubPlanState *node,
isNull,
isDone);
}
+
+/*
+ * ExecAndFormSerializeParamExec
+ *
+ * Execute the subplan stored in PARAM_EXEC param if it is not executed
+ * till now and form the serialized structure required for passing to
+ * worker backend.
+ */
+List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params)
+{
+ List *lparam = NIL;
+ SerializedParamExecData *sparamdata;
+ ParamExecData *prm;
+ int paramid;
+
+ paramid = -1;
+ while ((paramid = bms_next_member(params, paramid)) >= 0)
+ {
+ /*
+ * PARAM_EXEC params (internal executor parameters) are stored in the
+ * ecxt_param_exec_vals array, and can be accessed by array index.
+ */
+ sparamdata = palloc0(sizeof(SerializedParamExecData));
+
+ prm = &(econtext->ecxt_param_exec_vals[paramid]);
+ if (prm->execPlan != NULL)
+ {
+ /* Parameter not evaluated yet, so go do it */
+ ExecSetParamPlan(prm->execPlan, econtext);
+ /* ExecSetParamPlan should have processed this param... */
+ Assert(prm->execPlan == NULL);
+ }
+
+ sparamdata->paramid = paramid;
+ sparamdata->ptype = prm->ptype;
+ sparamdata->value = prm->value;
+ sparamdata->isnull = prm->isnull;
+
+ lparam = lappend(lparam, sparamdata);
+ }
+
+ return lparam;
+}
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 557d153..60cfab4 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -1753,7 +1753,7 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
* store tuple retrieved by Executor into SPITupleTable
* of current SPI procedure
*/
-void
+bool
spi_printtup(TupleTableSlot *slot, DestReceiver *self)
{
SPITupleTable *tuptable;
@@ -1787,6 +1787,8 @@ spi_printtup(TupleTableSlot *slot, DestReceiver *self)
(tuptable->free)--;
MemoryContextSwitchTo(oldcxt);
+
+ return true;
}
/*
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..2be48f4
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,304 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static bool
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result == SHM_MQ_DETACHED)
+ return false;
+ else if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+
+ return true;
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Detach all tuple queues that belong to funnel.
+ */
+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_from_handle(mqh);
+ shm_mq_detach(mq);
+ }
+ }
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/executor/tstoreReceiver.c b/src/backend/executor/tstoreReceiver.c
index c1fdeb7..b0862ae 100644
--- a/src/backend/executor/tstoreReceiver.c
+++ b/src/backend/executor/tstoreReceiver.c
@@ -37,8 +37,8 @@ typedef struct
} TStoreState;
-static void tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
-static void tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
/*
@@ -90,19 +90,21 @@ tstoreStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the easy case where we don't have to detoast.
*/
-static void
+static bool
tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the case where we have to detoast any toasted values.
*/
-static void
+static bool
tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
@@ -152,6 +154,8 @@ tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
/* And release any temporary detoasted values */
for (i = 0; i < nfree; i++)
pfree(DatumGetPointer(myState->tofree[i]));
+
+ return true;
}
/*
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index d8c9a0e..3c0123a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -355,6 +355,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4049,6 +4086,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 1aa1f55..05d4b3c 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -440,6 +440,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -2898,6 +2916,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..0050195 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, same as in original Param */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,355 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
+
+/*
+ * Estimate the amount of space required to serialize the PARAM_EXEC
+ * parameters.
+ */
+Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals)
+{
+ Size size;
+ ListCell *lparam;
+
+ /*
+ * Add space required for saving number of PARAM_EXEC parameters
+ * that needs to be serialized.
+ */
+ size = sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size length;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ length = sizeof(SerializedParamExecData);
+
+ get_typlenbyval(param_val->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || param_val->isnull))
+ {
+ length += datumGetSize(param_val->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the PARAM_EXEC parameters into the memory, beginning at
+ * start_address. maxsize should be at least as large as the value
+ * returned by EstimateExecParametersSpace.
+ */
+void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address)
+{
+ char *curptr;
+ SerializedParamExecData *retval;
+ ListCell *lparam;
+
+ /*
+ * First, we store the number of PARAM_EXEC parameters that needs to
+ * be serialized.
+ */
+ if (serialized_param_exec_vals)
+ * (int *) start_address = list_length(serialized_param_exec_vals);
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+
+ curptr = start_address + sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ retval = (SerializedParamExecData*) curptr;
+
+ retval->paramid = param_val->paramid;
+ retval->value = param_val->value;
+ retval->isnull = param_val->isnull;
+ retval->ptype = param_val->ptype;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(retval->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(retval->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(retval->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreExecParams
+ * Restore PARAM_EXEC parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+List *
+RestoreExecParams(char *start_address)
+{
+ List *lparamexecvals = NIL;
+ //Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExecData *nprm;
+ SerializedParamExecData *outparam;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExecData *) curptr;
+
+ outparam = palloc0(sizeof(SerializedParamExecData));
+
+ /* copy the parameter info */
+ outparam->isnull = nprm->isnull;
+ outparam->value = nprm->value;
+ outparam->paramid = nprm->paramid;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ outparam->value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+
+ lparamexecvals = lappend(lparamexecvals, outparam);
+ }
+
+ return lparamexecvals;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 563209c..f757e92 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1280,6 +1280,124 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlan
+ */
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+/*
+ * _readScan
+ */
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
+
+/*
+ * _readResult
+ */
+static Result *
+_readResult(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(Result);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_NODE_FIELD(resconstantqual);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1409,6 +1527,14 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
+ else if (MATCH("RESULT", 6))
+ return_value = _readResult();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 58d78e6..528727c 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -410,6 +410,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 1a0d358..874c272 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,9 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
+ * parallel_startup_cost Cost of starting up parallel workers
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +104,16 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
+double parallel_startup_cost = DEFAULT_PARALLEL_STARTUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -220,6 +228,55 @@ cost_seqscan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of scanning a relation parallely.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nWorkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * parallel sequiantial scan.
+ */
+ run_cost = run_cost / (nWorkers + 1);
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ startup_cost += parallel_startup_cost * nWorkers;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..949e79b
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,80 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ /*
+ * There should be atleast one page to scan for each worker.
+ */
+ if (parallel_seqscan_degree <= rel->pages)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = rel->pages;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index cb69c03..6dd43f3 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -58,6 +58,10 @@ static Material *create_material_plan(PlannerInfo *root, MaterialPath *best_path
static Plan *create_unique_plan(PlannerInfo *root, UniquePath *best_path);
static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -100,6 +104,12 @@ static List *order_qual_clauses(PlannerInfo *root, List *clauses);
static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -228,6 +238,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
switch (best_path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -267,6 +278,10 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
plan = create_unique_plan(root,
(UniquePath *) best_path);
break;
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path);
+ break;
default:
elog(ERROR, "unrecognized node type: %d",
(int) best_path->pathtype);
@@ -343,6 +358,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -546,6 +568,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
switch (path->pathtype)
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1133,6 +1157,107 @@ create_seqscan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ List *tlist;
+ RelOptInfo *rel = best_path->path.parent;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /*
+ * For table scans, rather than using the relation targetlist (which is
+ * only those Vars actually needed by the query), we prefer to generate a
+ * tlist containing all Vars in order. This will allow the executor to
+ * optimize away projection of the table tuples, if possible. (Note that
+ * planner.c may replace the tlist we generate here, forcing projection to
+ * occur.)
+ */
+ if (use_physical_tlist(root, rel))
+ {
+ tlist = build_physical_tlist(root, rel);
+ /* if fail because of dropped cols, use regular method */
+ if (tlist == NIL)
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+ else
+ {
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3321,6 +3446,45 @@ make_seqscan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 1824e7b..d16fa09 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -275,6 +275,52 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable,
+ int num_exec_params)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = num_exec_params;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 94b12ab..e26b248 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -435,6 +435,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -445,6 +446,26 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection. We don't want to do this assignment after
+ * fixing references as that will be done separately for partial
+ * sequence scan node.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
@@ -2062,6 +2083,45 @@ fix_opfuncids_walker(Node *node, void *context)
}
/*
+ * fix_node_funcids
+ * Set the opfuncid (procedure OID) in an OpExpr node,
+ * for plan tree.
+ *
+ * We need it mainly to fix the opfuncid in nodes of plantree
+ * after reading the planned statement by worker backend.
+ * Currently the support of nodes that could be executed by
+ * worker backend are limited, so we can enhance this API based
+ * on it's usage in future.
+ */
+void
+fix_node_funcids(Plan *node)
+{
+ /*
+ * do nothing when we get to the end of a leaf on tree.
+ */
+ if (node == NULL)
+ return;
+
+ fix_opfuncids((Node*) node->qual);
+ fix_opfuncids((Node*) node->targetlist);
+
+ switch (nodeTag(node))
+ {
+ case T_Result:
+ fix_opfuncids((Node*) (((Result *)node)->resconstantqual));
+ break;
+ case T_PartialSeqScan:
+ break;
+ default:
+ elog(ERROR, "unrecognized node type: %d", (int) nodeTag(node));
+ break;
+ }
+
+ fix_node_funcids(node->lefttree);
+ fix_node_funcids(node->righttree);
+}
+
+/*
* set_opfuncid
* Set the opfuncid (procedure OID) in an OpExpr node,
* if it hasn't been set already.
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index acfd0bc..f649639 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2167,6 +2167,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
break;
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index faca30b..0e5fd3a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -706,6 +706,53 @@ create_seqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* seqscan has unordered result */
+
+ cost_seqscan(pathnode, root, rel, pathnode->param_info);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nWorkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nWorkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info, nWorkers);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..f056bd5 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -12,7 +12,8 @@ subdir = src/backend/postmaster
top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
-OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+OBJS = autovacuum.o backendworker.o bgworker.o bgwriter.o checkpointer.o \
+ fork_process.o pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/backendworker.c b/src/backend/postmaster/backendworker.c
new file mode 100644
index 0000000..f4f6235
--- /dev/null
+++ b/src/backend/postmaster/backendworker.c
@@ -0,0 +1,471 @@
+/*-------------------------------------------------------------------------
+ *
+ * backendworker.c
+ * Support routines for setting up backend workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/postmaster/backendworker.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "executor/nodeFunnel.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "postmaster/backendworker.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+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);
+static void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+static shm_toc *parallel_shm_toc = NULL;
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters, PARAM_EXEC parameters and instrumentation
+ * information that need to be retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ List *serialized_param_exec_vals,
+ int instOptions, Size *params_size,
+ Size *params_exec_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ *params_exec_size = EstimateExecParametersSpace(serialized_param_exec_vals);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_exec_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 4);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters, PARAM_EXEC parameters and instrumentation
+ * information required for parallel execution.
+ */
+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)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Store PARAM_EXEC parameters list in dynamic shared memory. This is
+ * used for evaluation plan->initPlan params.
+ */
+ paramsexecdata = shm_toc_allocate(pcxt->toc, params_exec_size);
+ SerializeExecParams(serialized_param_exec_vals, params_exec_size, paramsexecdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS_EXEC, paramsexecdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePartialSeqScanSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel heap scan descriptor that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePartialSeqScanSpace(ParallelContext *pcxt, EState *estate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for partial seq. scan specific contents. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 2);
+}
+
+/*
+ * StorePartialSeqScan
+ *
+ * Sets up the planned statement and block range for parallel
+ * sequence scan.
+ */
+void
+StorePartialSeqScan(ParallelContext *pcxt, EState *estate, Relation rel,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+ ParallelHeapScanDesc pscan;
+
+ /* Store range table list in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, pscan_size);
+ heap_parallelscan_initialize(pscan, rel, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(Plan *plan, List *serialized_param_exec_vals,
+ EState *estate, Relation rel,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, params_exec_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)plan,
+ estate->es_range_table,
+ estate->es_plannedstmt->nParamExec);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePartialSeqScanSpace(pcxt, estate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument, ¶ms_size,
+ ¶ms_exec_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePartialSeqScan(pcxt, estate, rel, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument,
+ params_size,
+ params_exec_size,
+ inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameters, PARAM_EXEC parameters and
+ * instrumentation information required to perform parallel
+ * operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ paramsexecdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS_EXEC);
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+
+ *params = RestoreBoundParams(paramsdata);
+
+ *serialized_param_exec_vals = RestoreExecParams(paramsexecdata);
+
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+}
+
+/*
+ * ExecParallelGetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_node_funcids((*plannedstmt)->planTree);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * GetParallelShmToc
+ */
+shm_toc *
+GetParallelShmToc(void)
+{
+ return parallel_shm_toc;
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ ExecParallelGetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &serialized_param_exec_vals,
+ &inst_options, &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->serialized_param_exec_vals = serialized_param_exec_vals;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->responseq = responseq;
+
+ parallel_shm_toc = toc;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index a9f20ac..8a759a4 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/storage/ipc/shm_mq.c b/src/backend/storage/ipc/shm_mq.c
index d42a8d1..f640bb2 100644
--- a/src/backend/storage/ipc/shm_mq.c
+++ b/src/backend/storage/ipc/shm_mq.c
@@ -746,6 +746,15 @@ shm_mq_detach(shm_mq *mq)
}
/*
+ * Get the shm_mq from handle.
+ */
+shm_mq *
+shm_mq_from_handle(shm_mq_handle *mqh)
+{
+ return mqh->mqh_queue;
+}
+
+/*
* Write bytes into a shared message queue.
*/
static shm_mq_result
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..ba70bce 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -44,9 +45,10 @@
* dummy DestReceiver functions
* ----------------
*/
-static void
+static bool
donothingReceive(TupleTableSlot *slot, DestReceiver *self)
{
+ return true;
}
static void
@@ -129,6 +131,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +167,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +210,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +255,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7c18298..516c391 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,7 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -55,6 +56,7 @@
#include "pg_getopt.h"
#include "postmaster/autovacuum.h"
#include "postmaster/postmaster.h"
+#include "postmaster/backendworker.h"
#include "replication/slot.h"
#include "replication/walsender.h"
#include "rewrite/rewriteHandler.h"
@@ -1192,6 +1194,98 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ if (parallelstmt->inst_options)
+ receiver = None_Receiver;
+ else
+ {
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+ }
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ PopulateParamExecParams(queryDesc, parallelstmt->serialized_param_exec_vals);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins to report the total usage for statement execution.
+ */
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..f2fb638 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -1121,7 +1121,13 @@ RunFromStore(Portal portal, ScanDirection direction, long count,
if (!ok)
break;
- (*dest->receiveSlot) (slot, dest);
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
ExecClearTuple(slot);
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 8727ee3..0a10ebe 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -602,6 +602,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2551,6 +2553,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2738,6 +2750,36 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_startup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "starting parallel workers."),
+ NULL
+ },
+ ¶llel_startup_cost,
+ DEFAULT_PARALLEL_STARTUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 110983f..06c5969 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,9 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
+#parallel_startup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +504,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 888cce7..0a34b48 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -95,8 +95,9 @@ extern Relation heap_openrv_extended(const RangeVar *relation,
#define heap_close(r,l) relation_close(r,l)
-/* struct definition appears in relscan.h */
+/* struct definitions appear in relscan.h */
typedef struct HeapScanDescData *HeapScanDesc;
+typedef struct ParallelHeapScanDescData *ParallelHeapScanDesc;
/*
* HeapScanIsValid
@@ -116,9 +117,15 @@ extern HeapScanDesc heap_beginscan_bm(Relation relation, Snapshot snapshot,
extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
+extern Size heap_parallelscan_estimate(Snapshot snapshot);
+extern void heap_parallelscan_initialize(ParallelHeapScanDesc target,
+ Relation relation, Snapshot snapshot);
+extern HeapScanDesc heap_beginscan_parallel(Relation, ParallelHeapScanDesc);
+
extern bool heap_fetch(Relation relation, Snapshot snapshot,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
Relation stats_relation);
diff --git a/src/include/access/printtup.h b/src/include/access/printtup.h
index 46c4148..92ec882 100644
--- a/src/include/access/printtup.h
+++ b/src/include/access/printtup.h
@@ -25,11 +25,11 @@ extern void SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist,
extern void debugStartup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void debugtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool debugtup(TupleTableSlot *slot, DestReceiver *self);
/* XXX these are really in executor/spi.c */
extern void spi_dest_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void spi_printtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool spi_printtup(TupleTableSlot *slot, DestReceiver *self);
#endif /* PRINTTUP_H */
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 9bb6362..f459020 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,6 +20,15 @@
#include "access/itup.h"
#include "access/tupdesc.h"
+/* Struct for parallel scan setup */
+typedef struct ParallelHeapScanDescData
+{
+ Oid phs_relid;
+ BlockNumber phs_nblocks;
+ slock_t phs_mutex;
+ BlockNumber phs_cblock;
+ char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelHeapScanDescData;
typedef struct HeapScanDescData
{
@@ -48,6 +57,7 @@ typedef struct HeapScanDescData
BlockNumber rs_cblock; /* current block # in scan, if any */
Buffer rs_cbuf; /* current buffer in scan, if any */
/* NB: if rs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
+ ParallelHeapScanDesc rs_parallel; /* parallel scan information */
/* these fields only used in page-at-a-time mode and for bitmap scans */
int rs_cindex; /* current tuple's index in vistuples */
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index c1e7477..aef04a2 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -271,6 +271,8 @@ extern TupleDesc ExecCleanTypeFromTL(List *targetList, bool hasoid);
extern TupleDesc ExecTypeFromExprList(List *exprList);
extern void ExecTypeSetColNames(TupleDesc typeInfo, List *namesList);
extern void UpdateChangedParamSet(PlanState *node, Bitmapset *newchg);
+extern void PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals);
typedef struct TupOutputState
{
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index c9a2129..0c7847d 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..3af3a0e
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..cb05be7
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,24 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/nodeSubplan.h b/src/include/executor/nodeSubplan.h
index 3732ad4..21c745e 100644
--- a/src/include/executor/nodeSubplan.h
+++ b/src/include/executor/nodeSubplan.h
@@ -24,4 +24,7 @@ extern void ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent);
extern void ExecSetParamPlan(SubPlanState *node, ExprContext *econtext);
+extern List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params);
+
#endif /* NODESUBPLAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..d2ddb6e
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,35 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void TupleQueueFunnelShutdown(TupleQueueFunnel *funnel);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index ac75f86..cd79588 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
#include "utils/reltrigger.h"
@@ -389,6 +391,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1016,6 +1030,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1216,6 +1235,45 @@ typedef struct ScanState
typedef ScanState SeqScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 38469ef..3f3d572 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -97,6 +99,8 @@ typedef enum NodeTag
T_BitmapOrState,
T_ScanState,
T_SeqScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -217,6 +221,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..21c6f7a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -14,6 +14,8 @@
#ifndef PARAMS_H
#define PARAMS_H
+#include "nodes/pg_list.h"
+
/* To avoid including a pile of parser headers, reference ParseState thus: */
struct ParseState;
@@ -96,11 +98,47 @@ typedef struct ParamExecData
{
void *execPlan; /* should be "SubPlanState *" */
Datum value;
+ /*
+ * parameter's datatype, or 0. This is required so that
+ * datum value can be read and used for other purposes like
+ * passing it to worker backend via shared memory. This is
+ * required only for evaluation of initPlan's, however for
+ * consistency we set this for Subplan as well. We left it
+ * for other cases like CTE or RecursiveUnion cases where this
+ * structure is not used for evaluation of subplans.
+ */
+ Oid ptype;
bool isnull;
} ParamExecData;
+/*
+ * This structure is used to pass PARAM_EXEC parameters to backend
+ * workers. For each PARAM_EXEC parameter, pass this structure
+ * followed by value except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExecData
+{
+ int paramid; /* parameter id of this param */
+ Size length; /* length of parameter value */
+ Oid ptype; /* parameter's datatype, or 0 */
+ Datum value;
+ bool isnull;
+} SerializedParamExecData;
+
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
+extern Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals);
+extern void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address);
+List *
+RestoreExecParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/parsenodes.h b/src/include/nodes/parsenodes.h
index 812b7cf..61b943b 100644
--- a/src/include/nodes/parsenodes.h
+++ b/src/include/nodes/parsenodes.h
@@ -20,10 +20,14 @@
#ifndef PARSENODES_H
#define PARSENODES_H
+#include "executor/instrument.h"
#include "nodes/bitmapset.h"
#include "nodes/lockoptions.h"
+#include "nodes/params.h"
+#include "nodes/plannodes.h"
#include "nodes/primnodes.h"
#include "nodes/value.h"
+#include "storage/shm_mq.h"
/* Possible sources of a Query */
typedef enum QuerySource
@@ -156,6 +160,17 @@ typedef struct Query
* depends on to be semantically valid */
} Query;
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
/****************************************************************************
* Supporting data structures for Parse Trees
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 5f0ea1c..7cdf632 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -281,6 +281,22 @@ typedef struct Scan
typedef Scan SeqScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 32a5571..9689972 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -742,6 +742,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 9c2000b..11f0409 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,14 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
+#define DEFAULT_PARALLEL_STARTUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +56,12 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
+extern PGDLLIMPORT double parallel_startup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -68,6 +80,8 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
double index_pages, PlannerInfo *root);
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info, int nWorkers);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9923f0e..7873565 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -32,6 +32,11 @@ extern bool add_path_precheck(RelOptInfo *parent_rel,
extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nWorkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 6cad92e..391d519 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -46,6 +46,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index fa72918..c38b1e0 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -131,6 +131,7 @@ extern bool query_is_distinct_for(Query *query, List *colnos, List *opids);
*/
extern Plan *set_plan_references(PlannerInfo *root, Plan *plan);
extern void fix_opfuncids(Node *node);
+extern void fix_node_funcids(Plan *node);
extern void set_opfuncid(OpExpr *opexpr);
extern void set_sa_opfuncid(ScalarArrayOpExpr *opexpr);
extern void record_plan_function_dependency(PlannerInfo *root, Oid funcid);
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..8c7ce75 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable, int num_exec_params);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/postmaster/backendworker.h b/src/include/postmaster/backendworker.h
new file mode 100644
index 0000000..5ddd3c8
--- /dev/null
+++ b/src/include/postmaster/backendworker.h
@@ -0,0 +1,44 @@
+/*--------------------------------------------------------------------
+ * backendworker.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/postmaster/backendworker.h
+ *--------------------------------------------------------------------
+ */
+#ifndef BACKENDWORKER_H
+#define BACKENDWORKER_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_PARAMS_EXEC 2
+#define PARALLEL_KEY_BUFF_USAGE 3
+#define PARALLEL_KEY_INST_OPTIONS 4
+#define PARALLEL_KEY_INST_INFO 5
+#define PARALLEL_KEY_TUPLE_QUEUE 6
+#define PARALLEL_KEY_SCAN 7
+
+extern int parallel_seqscan_degree;
+
+extern void InitializeParallelWorkers(Plan *plan,
+ List *serialized_param_exec_vals,
+ EState *estate, Relation rel,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+extern shm_toc *GetParallelShmToc(void);
+
+#endif /* BACKENDWORKER_H */
diff --git a/src/include/storage/shm_mq.h b/src/include/storage/shm_mq.h
index 085a8a7..f94ebb8 100644
--- a/src/include/storage/shm_mq.h
+++ b/src/include/storage/shm_mq.h
@@ -65,6 +65,9 @@ extern void shm_mq_set_handle(shm_mq_handle *, BackgroundWorkerHandle *);
/* Break connection. */
extern void shm_mq_detach(shm_mq *);
+/* Get the shm_mq from handle. */
+extern shm_mq *shm_mq_from_handle(shm_mq_handle *mqh);
+
/* Send or receive messages. */
extern shm_mq_result shm_mq_send(shm_mq_handle *mqh,
Size nbytes, const void *data, bool nowait);
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..ff99d2c 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
@@ -103,7 +104,9 @@ typedef enum
* pointers that the executor must call.
*
* Note: the receiveSlot routine must be passed a slot containing a TupleDesc
- * identical to the one given to the rStartup routine.
+ * identical to the one given to the rStartup routine. It returns bool where
+ * a "true" value means "continue processing" and a "false" value means
+ * "stop early, just as if we'd reached the end of the scan".
* ----------------
*/
typedef struct _DestReceiver DestReceiver;
@@ -111,7 +114,7 @@ typedef struct _DestReceiver DestReceiver;
struct _DestReceiver
{
/* Called for each tuple to be output: */
- void (*receiveSlot) (TupleTableSlot *slot,
+ bool (*receiveSlot) (TupleTableSlot *slot,
DestReceiver *self);
/* Per-executor-run initialization and shutdown: */
void (*rStartup) (DestReceiver *self,
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 96c5b8b..33211d6 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -84,5 +84,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index cf319af..38855e5 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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 patchTo 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Apr 23, 2015 at 2:26 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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 patchTo 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.
Right, will fix as per suggestion.
- 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.
- shm_mq_from_handle() is probably reasonable, but can we rename it
shm_mq_get_queue()?
Okay, will change.
- 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?
- 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?
- 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.
- 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 bThe 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. 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).
- 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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 bThe 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Apr 28, 2015 at 5:37 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Apr 24, 2015 at 8:32 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.
- 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.
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.
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).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, May 6, 2015 at 7:10 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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())
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, May 18, 2015 at 6:28 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
wrote:
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
comments in this week and will post a new version.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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?
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Apr 23, 2015 at 2:26 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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 patchTo 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.
Okay, added new routine FinishParallelSetupAndAccumStats() which
will be called both from ExecEndFunnel() and when ExecFunnel() hits
the TupIsNull(slot) path. Apart from that the same routine is called
from some other paths like rescan and when we need to collect
statistics after execution is complete but still ExecEndFunnel() is
not called. This routine ensures that once it has collected the
stats of parallel workers and destroyed the parallel context, it will
do nothing on next execution unless the node is re-initialized.
- 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.
Okay, introduced the new function planstate_tree_walker(), so that
it can work for anything below funnel node.
- shm_mq_from_handle() is probably reasonable, but can we rename it
shm_mq_get_queue()?
Changed as per suggestion.
- 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.
You are right, I have removed this part of code.
- 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.
Changed postmaster/backendworkers.c to executor/execParallel.c
and moved ParallelStmt to executor/execParallel.h
- 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.
As per discussion, it makes sense to remove parallel_startup_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 bThe 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.
Okay, changed as per suggestion.
- 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.
Right, I have changed as per suggestion, but now it will only choose
the parallel path for bigger relations, so to test with smaller relations
one way is to reduce the cpu_tuple_comm_cost.
Note - You need to apply assess-parallel-safety-v5.patch (posted by
Robert on thread assessing parallel-safety) before this patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v15.patchapplication/octet-stream; name=parallel_seqscan_v15.patchDownload
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..639451a 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -26,9 +26,9 @@
static void printtup_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-static void printtup(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_20(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
static void printtup_shutdown(DestReceiver *self);
static void printtup_destroy(DestReceiver *self);
@@ -299,7 +299,7 @@ printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
* printtup --- print a tuple in protocol 3.0
* ----------------
*/
-static void
+static bool
printtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -376,13 +376,15 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
* printtup_20 --- print a tuple in protocol 2.0
* ----------------
*/
-static void
+static bool
printtup_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -452,6 +454,8 @@ printtup_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
@@ -528,7 +532,7 @@ debugStartup(DestReceiver *self, int operation, TupleDesc typeinfo)
* debugtup - print one tuple for an interactive backend
* ----------------
*/
-void
+bool
debugtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -553,6 +557,8 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
printatt((unsigned) i + 1, typeinfo->attrs[i], value);
}
printf("\t----\n");
+
+ return true;
}
/* ----------------
@@ -564,7 +570,7 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
* This is largely same as printtup_20, except we use binary formatting.
* ----------------
*/
-static void
+static bool
printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -636,4 +642,6 @@ printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb86a4f..9324b7e 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -63,6 +63,7 @@
#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
+#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/datum.h"
#include "utils/inval.h"
@@ -80,9 +81,11 @@ bool synchronize_seqscans = true;
static HeapScanDesc heap_beginscan_internal(Relation relation,
Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync, bool allow_pagemode,
bool is_bitmapscan, bool is_samplescan,
bool temp_snap);
+static BlockNumber heap_parallelscan_nextpage(ParallelHeapScanDesc);
static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
TransactionId xid, CommandId cid, int options);
static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
@@ -223,7 +226,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool is_rescan)
* results for a non-MVCC snapshot, the caller must hold some higher-level
* lock that ensures the interesting tuple(s) won't change.)
*/
- scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
+ if (scan->rs_parallel != NULL)
+ scan->rs_nblocks = scan->rs_parallel->phs_nblocks;
+ else
+ scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
/*
* If the table is large relative to NBuffers, use a bulk-read access
@@ -483,7 +489,18 @@ heapgettup(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineoff = FirstOffsetNumber; /* first offnum */
scan->rs_inited = true;
@@ -506,6 +523,9 @@ heapgettup(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -658,11 +678,19 @@ heapgettup(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -760,7 +788,18 @@ heapgettup_pagemode(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineindex = 0;
scan->rs_inited = true;
@@ -780,6 +819,9 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -921,11 +963,19 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -1321,7 +1371,7 @@ HeapScanDesc
heap_beginscan(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, false);
}
@@ -1331,7 +1381,7 @@ heap_beginscan_catalog(Relation relation, int nkeys, ScanKey key)
Oid relid = RelationGetRelid(relation);
Snapshot snapshot = RegisterSnapshot(GetCatalogSnapshot(relid));
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, true);
}
@@ -1340,7 +1390,7 @@ heap_beginscan_strat(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, true,
false, false, false);
}
@@ -1349,7 +1399,7 @@ HeapScanDesc
heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
false, false, true, true, false, false);
}
@@ -1358,7 +1408,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_pagemode)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, false, allow_pagemode,
false, true, false);
}
@@ -1366,6 +1416,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
static HeapScanDesc
heap_beginscan_internal(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync, bool allow_pagemode,
bool is_bitmapscan, bool is_samplescan, bool temp_snap)
{
@@ -1394,6 +1445,7 @@ heap_beginscan_internal(Relation relation, Snapshot snapshot,
scan->rs_allow_strat = allow_strat;
scan->rs_allow_sync = allow_sync;
scan->rs_temp_snap = temp_snap;
+ scan->rs_parallel = parallel_scan;
/*
* we can use page-at-a-time mode if it's an MVCC-safe snapshot
@@ -1487,6 +1539,94 @@ heap_endscan(HeapScanDesc scan)
}
/* ----------------
+ * heap_parallelscan_estimate - estimate storage for ParallelHeapScanDesc
+ *
+ * Sadly, this doesn't reduce to a constant, because the size required
+ * to serialize the snapshot can vary.
+ * ----------------
+ */
+Size
+heap_parallelscan_estimate(Snapshot snapshot)
+{
+ return add_size(offsetof(ParallelHeapScanDescData, phs_snapshot_data),
+ EstimateSnapshotSpace(snapshot));
+}
+
+/* ----------------
+ * heap_parallelscan_initialize - initialize ParallelHeapScanDesc
+ *
+ * Must allow as many bytes of shared memory as returned by
+ * heap_parallelscan_estimate. Call this just once in the leader
+ * process; then, individual workers attach via heap_beginscan_parallel.
+ * ----------------
+ */
+void
+heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
+ Snapshot snapshot)
+{
+ target->phs_relid = RelationGetRelid(relation);
+ target->phs_nblocks = RelationGetNumberOfBlocks(relation);
+ SpinLockInit(&target->phs_mutex);
+ target->phs_cblock = 0;
+ SerializeSnapshot(snapshot, target->phs_snapshot_data);
+}
+/* ----------------
+ * heap_parallelscan_nextpage - get the next page to scan
+ *
+ * A return value larger than the number of blocks to be scanned
+ * indicates end of scan. Note, however, that other backends could still
+ * be scanning if they grabbed a page to scan and aren't done with it yet.
+ * ----------------
+ */
+static BlockNumber
+heap_parallelscan_nextpage(ParallelHeapScanDesc parallel_scan)
+{
+ BlockNumber page = InvalidBlockNumber;
+
+ /* we treat InvalidBlockNumber specially here to avoid overflow */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock != InvalidBlockNumber)
+ page = parallel_scan->phs_cblock++;
+ SpinLockRelease(¶llel_scan->phs_mutex);
+
+ return page;
+}
+
+/* ----------------
+ * heap_beginscan_parallel - join a parallel scan
+ *
+ * Caller must hold a suitable lock on the correct relation.
+ * ----------------
+ */
+HeapScanDesc
+heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
+{
+ Snapshot snapshot;
+
+ Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
+ snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
+ RegisterSnapshot(snapshot);
+
+ return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
+ true, true, true, false, false, true);
+}
+
+/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/catalog/system_views.sql b/src/backend/catalog/system_views.sql
index 18921c4..967672a 100644
--- a/src/backend/catalog/system_views.sql
+++ b/src/backend/catalog/system_views.sql
@@ -671,7 +671,7 @@ CREATE VIEW pg_replication_slots AS
L.datoid,
D.datname AS database,
L.active,
- L.active_pid,
+ L.active_pid,
L.xmin,
L.catalog_xmin,
L.restart_lsn
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 3e14c53..aa10678 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -4394,7 +4394,7 @@ copy_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* copy_dest_receive --- receive one tuple
*/
-static void
+static bool
copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_copy *myState = (DR_copy *) self;
@@ -4406,6 +4406,8 @@ copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
/* And send the data */
CopyOneRowTo(cstate, InvalidOid, slot->tts_values, slot->tts_isnull);
myState->processed++;
+
+ return true;
}
/*
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index e8f0d79..2eac70e 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -62,7 +62,7 @@ typedef struct
static ObjectAddress CreateAsReladdr = {InvalidOid, InvalidOid, 0};
static void intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void intorel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool intorel_receive(TupleTableSlot *slot, DestReceiver *self);
static void intorel_shutdown(DestReceiver *self);
static void intorel_destroy(DestReceiver *self);
@@ -482,7 +482,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* intorel_receive --- receive one tuple
*/
-static void
+static bool
intorel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_intorel *myState = (DR_intorel *) self;
@@ -507,6 +507,8 @@ intorel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 9d47308..4b98aaa 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -20,6 +20,7 @@
#include "commands/defrem.h"
#include "commands/prepare.h"
#include "executor/hashjoin.h"
+#include "executor/nodeFunnel.h"
#include "foreign/fdwapi.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
@@ -728,6 +729,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -933,6 +936,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1098,6 +1107,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1245,6 +1256,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. Though we already accumulate this
+ * information when last tuple is fetched from Funnel node, this
+ * is to cover cases when we don't fetch all tuples from a node
+ * such as for Limit node.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ FinishParallelSetupAndAccumStats((FunnelState *)planstate);
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1361,6 +1382,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1371,6 +1393,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2357,6 +2387,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index eb16bb3..78f822b 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -56,7 +56,7 @@ typedef struct
static int matview_maintenance_depth = 0;
static void transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
static void transientrel_shutdown(DestReceiver *self);
static void transientrel_destroy(DestReceiver *self);
static void refresh_matview_datafill(DestReceiver *dest, Query *query,
@@ -422,7 +422,7 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* transientrel_receive --- receive one tuple
*/
-static void
+static bool
transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_transientrel *myState = (DR_transientrel *) self;
@@ -441,6 +441,8 @@ transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index 08cba6f..be1f47e 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -13,17 +13,17 @@ top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = execAmi.o execCurrent.o execGrouping.o execIndexing.o execJunk.o \
- execMain.o execProcnode.o execQual.o execScan.o execTuples.o \
+ execMain.o execParallel.o execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
- nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeNestloop.o nodeFunctionscan.o nodePartialSeqscan.o nodeRecursiveunion.o \
+ nodeResult.o nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 4948a26..7f9baa6 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -160,6 +162,14 @@ ExecReScan(PlanState *node)
ExecReScanSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -463,6 +473,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index bcd287f..7a44462 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -262,6 +262,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
*/
case T_SeqScanState:
case T_SampleScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index aefc9fa..a6417ef 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -45,9 +45,11 @@
#include "commands/matview.h"
#include "commands/trigger.h"
#include "executor/execdebug.h"
+#include "executor/execParallel.h"
#include "foreign/fdwapi.h"
#include "mb/pg_wchar.h"
#include "miscadmin.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "parser/parsetree.h"
#include "storage/bufmgr.h"
@@ -323,6 +325,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
@@ -354,7 +359,15 @@ standard_ExecutorRun(QueryDesc *queryDesc,
(*dest->rShutdown) (dest);
if (queryDesc->totaltime)
+ {
+ /*
+ * Accumulate the stats by parallel workers before stopping the
+ * node.
+ */
+ (void) planstate_tree_walker((Node*) queryDesc->planstate,
+ NULL, ExecParallelBufferUsageAccum, 0);
InstrStopNode(queryDesc->totaltime, estate->es_processed);
+ }
MemoryContextSwitchTo(oldcontext);
}
@@ -1582,7 +1595,15 @@ ExecutePlan(EState *estate,
* practice, this is probably always the case at this point.)
*/
if (sendTuples)
- (*dest->receiveSlot) (slot, dest);
+ {
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
+ }
/*
* Count tuples processed, if this is a SELECT. (For other operation
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
new file mode 100644
index 0000000..33e83fe
--- /dev/null
+++ b/src/backend/executor/execParallel.c
@@ -0,0 +1,592 @@
+/*-------------------------------------------------------------------------
+ *
+ * execParallel.c
+ * Support routines for setting up backend workers for parallel execution.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/execParallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodePartialSeqscan.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+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);
+static void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+static shm_toc *parallel_shm_toc = NULL;
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters, PARAM_EXEC parameters and instrumentation
+ * information that need to be retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ List *serialized_param_exec_vals,
+ int instOptions, Size *params_size,
+ Size *params_exec_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ *params_exec_size = EstimateExecParametersSpace(serialized_param_exec_vals);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_exec_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 4);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters, PARAM_EXEC parameters and instrumentation
+ * information required for parallel execution.
+ */
+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)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Store PARAM_EXEC parameters list in dynamic shared memory. This is
+ * used for evaluation plan->initPlan params.
+ */
+ paramsexecdata = shm_toc_allocate(pcxt->toc, params_exec_size);
+ SerializeExecParams(serialized_param_exec_vals, params_exec_size, paramsexecdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS_EXEC, paramsexecdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePlannedStmtSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel node specific information that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for planned statement. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ /* keys for planned statement information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelEstimate,
+ pscan_size);
+}
+
+/*
+ * StorePlannedStmt
+ *
+ * Sets up the planned statement and node specific information.
+ */
+void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+
+ /* Store planned statement in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelInitializeDSM,
+ &pscan_size);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * ExecParallelEstimate
+ *
+ * Estimate the amount of space required to record information of
+ * parallel node that need to be copied to parallel workers.
+ */
+bool
+ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ if (node == NULL)
+ return false;
+
+ switch (nodeTag(node))
+ {
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelEstimate, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* key for paratial scan information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ return true;
+ }
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * ExecParallelInitializeDSM
+ *
+ * Store the information of parallel node in dsm.
+ */
+bool
+ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ ParallelHeapScanDesc pscan;
+
+ if (node == NULL)
+ return false;
+
+ switch (nodeTag(node))
+ {
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelInitializeDSM, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, *pscan_size);
+ heap_parallelscan_initialize(pscan, ((PartialSeqScanState*)node)->ss.ss_currentRelation, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+ return true;
+ }
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, params_exec_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)planstate->plan,
+ estate->es_range_table,
+ estate->es_plannedstmt->nParamExec);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePlannedStmtSpace(pcxt, planstate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument, ¶ms_size,
+ ¶ms_exec_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePlannedStmt(pcxt, planstate, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument,
+ params_size,
+ params_exec_size,
+ inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameters, PARAM_EXEC parameters and
+ * instrumentation information required to perform parallel
+ * operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ if (params)
+ {
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ *params = RestoreBoundParams(paramsdata);
+ }
+
+ if (serialized_param_exec_vals)
+ {
+ paramsexecdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS_EXEC);
+ *serialized_param_exec_vals = RestoreExecParams(paramsexecdata);
+ }
+
+ if (inst_options)
+ {
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+ }
+
+ if (buffer_usage)
+ {
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+ }
+}
+
+/*
+ * ExecParallelGetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_node_funcids((*plannedstmt)->planTree);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * GetParallelShmToc
+ */
+shm_toc *
+GetParallelShmToc(void)
+{
+ return parallel_shm_toc;
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ ExecParallelGetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &serialized_param_exec_vals,
+ &inst_options, &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->serialized_param_exec_vals = serialized_param_exec_vals;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->responseq = responseq;
+
+ parallel_shm_toc = toc;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
+
+/*
+ * ExecParallelBufferUsageAccum
+ *
+ * Recursively accumulate the stats for all the funnel nodes
+ * in a plan state tree.
+ */
+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;
+}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 03c2feb..e24a439 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -196,6 +198,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -416,6 +428,14 @@ ExecProcNode(PlanState *node)
result = ExecSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -658,6 +678,14 @@ ExecEndNode(PlanState *node)
ExecEndSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index a05d8b1..d5619bd 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -1313,7 +1313,7 @@ do_tup_output(TupOutputState *tstate, Datum *values, bool *isnull)
ExecStoreVirtualTuple(slot);
/* send the tuple to the receiver */
- (*tstate->dest->receiveSlot) (slot, tstate->dest);
+ (void) (*tstate->dest->receiveSlot) (slot, tstate->dest);
/* clean up */
ExecClearTuple(slot);
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 3963408..adf7439 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -974,3 +974,28 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
MemoryContextSwitchTo(oldcontext);
}
+
+/*
+ * Populate the values of PARAM_EXEC parameters.
+ *
+ * This is used by worker backends to fill in the values
+ * of PARAM_EXEC parameters after fetching the same from
+ * dynamic shared memory. This needs to be called before
+ * ExecutorRun.
+ */
+void
+PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals)
+{
+ ListCell *lparam;
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].value =
+ param_val->value;
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].isnull =
+ param_val->isnull;
+ }
+}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 812a610..863bd64 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -167,7 +167,7 @@ static Datum postquel_get_single_result(TupleTableSlot *slot,
static void sql_exec_error_callback(void *arg);
static void ShutdownSQLFunction(Datum arg);
static void sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
static void sqlfunction_shutdown(DestReceiver *self);
static void sqlfunction_destroy(DestReceiver *self);
@@ -1903,7 +1903,7 @@ sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* sqlfunction_receive --- receive one tuple
*/
-static void
+static bool
sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_sqlfunction *myState = (DR_sqlfunction *) self;
@@ -1913,6 +1913,8 @@ sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
/* Store the filtered tuple into the tuplestore */
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..3c42f21
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,436 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodeSubplan.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+static void ExecAccumulateInstInfo(FunnelState *node);
+static void ExecAccumulateBufUsageInfo(FunnelState *node);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ ExprContext *econtext = node->ss.ps.ps_ExprContext;
+ bool any_worker_launched = false;
+ List *serialized_param_exec;
+
+ /*
+ * 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);
+
+ /* Initialize the workers required to execute funnel node. */
+ InitializeParallelWorkers(node->ss.ps.lefttree,
+ serialized_param_exec,
+ estate,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * If there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ if (TupIsNull(slot))
+ {
+
+ /*
+ * Destroy the parallel context once we complete fetching all
+ * the tuples, this will ensure that if in the same statement
+ * we need to have Funnel node for multiple parts of statement,
+ * it won't accumulate lot of dsm's and workers can be made
+ * available to use by other parts of statement.
+ */
+ FinishParallelSetupAndAccumStats(node);
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ FinishParallelSetupAndAccumStats(node);
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * FinishParallelSetupAndAccumStats
+ *
+ * Destroy the setup for parallel workers. Collect all the
+ * stats after workers are stopped, else some work done by
+ * workers won't be accounted.
+ * ----------------------------------------------------------------
+ */
+void
+FinishParallelSetupAndAccumStats(FunnelState *node)
+{
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ {
+ TupleQueueFunnelShutdown(node->funnel);
+ WaitForParallelWorkersToFinish(node->pcxt);
+ }
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+ node->funnel = NULL;
+
+ /*
+ * Aggregate the buffer usage stats from all workers. This is
+ * required by external modules like pg_stat_statements.
+ */
+ ExecAccumulateBufUsageInfo(node);
+
+ /*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. This has to be done before we
+ * destroy the parallel context.
+ */
+ if (node->ss.ps.state->es_instrument)
+ ExecAccumulateInstInfo(node);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateInstInfo
+ *
+ * Accumulate instrumentation information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateInstInfo(FunnelState *node)
+{
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers;
+ char *inst_info_workers;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ inst_info_workers = node->inst_options_space;
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(node->ss.ps.instrument, instrument_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateBufUsageInfo
+ *
+ * Accumulate buffer usage information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateBufUsageInfo(FunnelState *node)
+{
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * 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);
+
+ ExecReScan(node->ss.ps.lefttree);
+}
diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c
index e66bcda..c447062 100644
--- a/src/backend/executor/nodeNestloop.c
+++ b/src/backend/executor/nodeNestloop.c
@@ -144,6 +144,7 @@ ExecNestLoop(NestLoopState *node)
{
NestLoopParam *nlp = (NestLoopParam *) lfirst(lc);
int paramno = nlp->paramno;
+ TupleDesc tdesc = outerTupleSlot->tts_tupleDescriptor;
ParamExecData *prm;
prm = &(econtext->ecxt_param_exec_vals[paramno]);
@@ -154,6 +155,7 @@ ExecNestLoop(NestLoopState *node)
prm->value = slot_getattr(outerTupleSlot,
nlp->paramval->varattno,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[nlp->paramval->varattno-1]->atttypid;
/* Flag parameter value as changed */
innerPlan->chgParam = bms_add_member(innerPlan->chgParam,
paramno);
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..09b7e07
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,308 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodePartialSeqscan.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ shm_toc *toc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We set 'toc' (place to lookup parallel scan
+ * descriptor) as retrievied by attaching to dsm for parallel workers
+ * whereas master backend stores it directly in partial scan state node
+ * after initializing workers.
+ */
+ toc = GetParallelShmToc();
+ if (toc)
+ node->ss.ps.toc = toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ node->scan_initialized = false;
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c
index 8d3dde0..b348bfd 100644
--- a/src/backend/executor/nodeResult.c
+++ b/src/backend/executor/nodeResult.c
@@ -75,6 +75,13 @@ ExecResult(ResultState *node)
econtext = node->ps.ps_ExprContext;
/*
+ * Result node can be added as a gating node on top of PartialSeqScan
+ * node, so need to percolate toc information to outer node.
+ */
+ if (node->ps.toc)
+ outerPlanState(node)->toc = node->ps.toc;
+
+ /*
* check constant qualifications like (2 > 1), if not already done
*/
if (node->rs_checkqual)
diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c
index 9eb4d63..6afd55a 100644
--- a/src/backend/executor/nodeSubplan.c
+++ b/src/backend/executor/nodeSubplan.c
@@ -30,11 +30,14 @@
#include <math.h>
#include "access/htup_details.h"
+#include "catalog/pg_type.h"
#include "executor/executor.h"
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "utils/array.h"
+#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
@@ -281,12 +284,14 @@ ExecScanSubPlan(SubPlanState *node,
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -399,6 +404,7 @@ ExecScanSubPlan(SubPlanState *node,
prmdata = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col, &(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
@@ -551,6 +557,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
!TupIsNull(slot);
slot = ExecProcNode(planstate))
{
+ TupleDesc tdesc = slot->tts_tupleDescriptor;
int col = 1;
ListCell *plst;
bool isnew;
@@ -568,6 +575,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col,
&(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
slot = ExecProject(node->projRight, NULL);
@@ -954,6 +962,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
ListCell *l;
bool found = false;
ArrayBuildStateAny *astate = NULL;
+ Oid ptype;
if (subLinkType == ANY_SUBLINK ||
subLinkType == ALL_SUBLINK)
@@ -961,6 +970,8 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (subLinkType == CTE_SUBLINK)
elog(ERROR, "CTE subplans should not be executed via ExecSetParamPlan");
+ ptype = exprType((Node *) node->xprstate.expr);
+
/* Initialize ArrayBuildStateAny in caller's context, if needed */
if (subLinkType == ARRAY_SUBLINK)
astate = initArrayResultAny(subplan->firstColType,
@@ -983,12 +994,14 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -1011,6 +1024,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(true);
+ prm->ptype = ptype;
prm->isnull = false;
found = true;
break;
@@ -1062,6 +1076,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = heap_getattr(node->curTuple, i, tdesc,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[i-1]->atttypid;
i++;
}
}
@@ -1084,6 +1099,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
true);
prm->execPlan = NULL;
prm->value = node->curArray;
+ prm->ptype = ptype;
prm->isnull = false;
}
else if (!found)
@@ -1096,6 +1112,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(false);
+ prm->ptype = ptype;
prm->isnull = false;
}
else
@@ -1108,6 +1125,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = (Datum) 0;
+ prm->ptype = VOIDOID;
prm->isnull = true;
}
}
@@ -1238,3 +1256,47 @@ ExecAlternativeSubPlan(AlternativeSubPlanState *node,
isNull,
isDone);
}
+
+/*
+ * ExecAndFormSerializeParamExec
+ *
+ * Execute the subplan stored in PARAM_EXEC param if it is not executed
+ * till now and form the serialized structure required for passing to
+ * worker backend.
+ */
+List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params)
+{
+ List *lparam = NIL;
+ SerializedParamExecData *sparamdata;
+ ParamExecData *prm;
+ int paramid;
+
+ paramid = -1;
+ while ((paramid = bms_next_member(params, paramid)) >= 0)
+ {
+ /*
+ * PARAM_EXEC params (internal executor parameters) are stored in the
+ * ecxt_param_exec_vals array, and can be accessed by array index.
+ */
+ sparamdata = palloc0(sizeof(SerializedParamExecData));
+
+ prm = &(econtext->ecxt_param_exec_vals[paramid]);
+ if (prm->execPlan != NULL)
+ {
+ /* Parameter not evaluated yet, so go do it */
+ ExecSetParamPlan(prm->execPlan, econtext);
+ /* ExecSetParamPlan should have processed this param... */
+ Assert(prm->execPlan == NULL);
+ }
+
+ sparamdata->paramid = paramid;
+ sparamdata->ptype = prm->ptype;
+ sparamdata->value = prm->value;
+ sparamdata->isnull = prm->isnull;
+
+ lparam = lappend(lparam, sparamdata);
+ }
+
+ return lparam;
+}
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 472de41..0f7906b 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -1774,7 +1774,7 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
* store tuple retrieved by Executor into SPITupleTable
* of current SPI procedure
*/
-void
+bool
spi_printtup(TupleTableSlot *slot, DestReceiver *self)
{
SPITupleTable *tuptable;
@@ -1808,6 +1808,8 @@ spi_printtup(TupleTableSlot *slot, DestReceiver *self)
(tuptable->free)--;
MemoryContextSwitchTo(oldcxt);
+
+ return true;
}
/*
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..39acda7
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,304 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static bool
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result == SHM_MQ_DETACHED)
+ return false;
+ else if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+
+ return true;
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Detach all tuple queues that belong to funnel.
+ */
+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);
+ }
+ }
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/executor/tstoreReceiver.c b/src/backend/executor/tstoreReceiver.c
index c1fdeb7..b0862ae 100644
--- a/src/backend/executor/tstoreReceiver.c
+++ b/src/backend/executor/tstoreReceiver.c
@@ -37,8 +37,8 @@ typedef struct
} TStoreState;
-static void tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
-static void tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
/*
@@ -90,19 +90,21 @@ tstoreStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the easy case where we don't have to detoast.
*/
-static void
+static bool
tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the case where we have to detoast any toasted values.
*/
-static void
+static bool
tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
@@ -152,6 +154,8 @@ tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
/* And release any temporary detoasted values */
for (i = 0; i < nfree; i++)
pfree(DatumGetPointer(myState->tofree[i]));
+
+ return true;
}
/*
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index ab03888..a200eca 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -362,6 +362,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4227,6 +4264,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index 4176393..8944195 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -3395,3 +3395,25 @@ raw_expression_tree_walker(Node *node,
}
return false;
}
+
+/*
+ * planstate_tree_walker
+ *
+ * This routine will invoke walker on the node passed. This is a useful
+ * way of starting the recursion when the walker's normal change of state
+ * is not appropriate for the outermost PlanState node.
+ */
+bool
+planstate_tree_walker(Node *node,
+ ParallelContext *pcxt,
+ bool (*walker) (),
+ void *context)
+{
+ if (node == NULL)
+ return false;
+
+ /* Guard against stack overflow due to overly complex plan */
+ check_stack_depth();
+
+ return walker(node, pcxt, context);
+}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 01ae278..429d017 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -447,6 +447,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -3005,6 +3023,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..0050195 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, same as in original Param */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,355 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
+
+/*
+ * Estimate the amount of space required to serialize the PARAM_EXEC
+ * parameters.
+ */
+Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals)
+{
+ Size size;
+ ListCell *lparam;
+
+ /*
+ * Add space required for saving number of PARAM_EXEC parameters
+ * that needs to be serialized.
+ */
+ size = sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size length;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ length = sizeof(SerializedParamExecData);
+
+ get_typlenbyval(param_val->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || param_val->isnull))
+ {
+ length += datumGetSize(param_val->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the PARAM_EXEC parameters into the memory, beginning at
+ * start_address. maxsize should be at least as large as the value
+ * returned by EstimateExecParametersSpace.
+ */
+void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address)
+{
+ char *curptr;
+ SerializedParamExecData *retval;
+ ListCell *lparam;
+
+ /*
+ * First, we store the number of PARAM_EXEC parameters that needs to
+ * be serialized.
+ */
+ if (serialized_param_exec_vals)
+ * (int *) start_address = list_length(serialized_param_exec_vals);
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+
+ curptr = start_address + sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ retval = (SerializedParamExecData*) curptr;
+
+ retval->paramid = param_val->paramid;
+ retval->value = param_val->value;
+ retval->isnull = param_val->isnull;
+ retval->ptype = param_val->ptype;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(retval->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(retval->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(retval->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreExecParams
+ * Restore PARAM_EXEC parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+List *
+RestoreExecParams(char *start_address)
+{
+ List *lparamexecvals = NIL;
+ //Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExecData *nprm;
+ SerializedParamExecData *outparam;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExecData *) curptr;
+
+ outparam = palloc0(sizeof(SerializedParamExecData));
+
+ /* copy the parameter info */
+ outparam->isnull = nprm->isnull;
+ outparam->value = nprm->value;
+ outparam->paramid = nprm->paramid;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ outparam->value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+
+ lparamexecvals = lappend(lparamexecvals, outparam);
+ }
+
+ return lparamexecvals;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index f5a40fb..4749efe 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -29,6 +29,7 @@
#include <math.h>
#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
#include "nodes/readfuncs.h"
@@ -1391,6 +1392,125 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(isUpsert);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlan
+ */
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+/*
+ * _readScan
+ */
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
+
+/*
+ * _readResult
+ */
+static Result *
+_readResult(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(Result);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_NODE_FIELD(resconstantqual);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1532,6 +1652,14 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
+ else if (MATCH("RESULT", 6))
+ return_value = _readResult();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 1fd8763..6f45da6 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -424,6 +424,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index c2b2b76..dec2853 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,8 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +103,15 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -287,6 +293,86 @@ cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel)
}
/*
+ * cost_patialseqscan
+ * Determines and returns the cost of scanning a relation partially.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ * 'nworkers' are the number of workers among which the work will be
+ * distributed
+ */
+void
+cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ cost_seqscan(path, root, baserel, param_info);
+
+ startup_cost = path->startup_cost;
+
+ run_cost = path->total_cost - startup_cost;
+
+ /*
+ * Account for small cost for communication related to scan
+ * via the ParallelHeapScanDesc.
+ */
+ run_cost += 0.01;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * partial sequiantial scan.
+ */
+ run_cost = run_cost / (nworkers + 1);
+
+ path->startup_cost = startup_cost;
+ path->total_cost = startup_cost + run_cost;
+}
+
+/*
+ * cost_funnel
+ * Determines and returns the cost of funnel path.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..bc71737
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,89 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ int estimated_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ /*
+ * There should be atleast thousand pages to scan for each worker.
+ * This number is somewhat arbitratry, however we don't want to
+ * spawn workers to scan smaller relations as that will be costly.
+ */
+ estimated_parallel_workers = rel->pages / 1000;
+
+ if (estimated_parallel_workers <= 0)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ if (parallel_seqscan_degree <= estimated_parallel_workers)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = estimated_parallel_workers;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false,
+ num_parallel_workers);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index b47ef46..4a57716 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -60,6 +60,10 @@ static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
static SampleScan *create_samplescan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -103,6 +107,11 @@ static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
static SampleScan *make_samplescan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist, List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -233,6 +242,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -272,6 +282,10 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
plan = create_unique_plan(root,
(UniquePath *) best_path);
break;
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path);
+ break;
default:
elog(ERROR, "unrecognized node type: %d",
(int) best_path->pathtype);
@@ -355,6 +369,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -559,6 +580,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1186,6 +1209,107 @@ create_samplescan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ List *tlist;
+ RelOptInfo *rel = best_path->path.parent;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /*
+ * For table scans, rather than using the relation targetlist (which is
+ * only those Vars actually needed by the query), we prefer to generate a
+ * tlist containing all Vars in order. This will allow the executor to
+ * optimize away projection of the table tuples, if possible. (Note that
+ * planner.c may replace the tlist we generate here, forcing projection to
+ * occur.)
+ */
+ if (use_physical_tlist(root, rel))
+ {
+ tlist = build_physical_tlist(root, rel);
+ /* if fail because of dropped cols, use regular method */
+ if (tlist == NIL)
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+ else
+ {
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3441,6 +3565,45 @@ make_samplescan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 70c2fcf..1cc2cdf 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -295,6 +295,52 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable,
+ int num_exec_params)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = num_exec_params;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 90e13e4..c611e30 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -440,6 +440,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -461,6 +462,26 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection. We don't want to do this assignment after
+ * fixing references as that will be done separately for partial
+ * sequence scan node.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
@@ -2251,6 +2272,45 @@ fix_opfuncids_walker(Node *node, void *context)
}
/*
+ * fix_node_funcids
+ * Set the opfuncid (procedure OID) in an OpExpr node,
+ * for plan tree.
+ *
+ * We need it mainly to fix the opfuncid in nodes of plantree
+ * after reading the planned statement by worker backend.
+ * Currently the support of nodes that could be executed by
+ * worker backend are limited, so we can enhance this API based
+ * on it's usage in future.
+ */
+void
+fix_node_funcids(Plan *node)
+{
+ /*
+ * do nothing when we get to the end of a leaf on tree.
+ */
+ if (node == NULL)
+ return;
+
+ fix_opfuncids((Node*) node->qual);
+ fix_opfuncids((Node*) node->targetlist);
+
+ switch (nodeTag(node))
+ {
+ case T_Result:
+ fix_opfuncids((Node*) (((Result *)node)->resconstantqual));
+ break;
+ case T_PartialSeqScan:
+ break;
+ default:
+ elog(ERROR, "unrecognized node type: %d", (int) nodeTag(node));
+ break;
+ }
+
+ fix_node_funcids(node->lefttree);
+ fix_node_funcids(node->righttree);
+}
+
+/*
* set_opfuncid
* Set the opfuncid (procedure OID) in an OpExpr node,
* if it hasn't been set already.
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index f80abb4..0fe3ac3 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2217,6 +2217,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 3fe2712..5098e70 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -726,6 +726,54 @@ create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* partialseqscan has unordered result */
+
+ cost_patialseqscan(pathnode, root, rel, pathnode->param_info, nworkers);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nworkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nworkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..4aec92a 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -13,6 +13,7 @@ top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+ pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 87f5430..c0f09ab 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -835,6 +836,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/storage/ipc/shm_mq.c b/src/backend/storage/ipc/shm_mq.c
index daca634..5f0d367 100644
--- a/src/backend/storage/ipc/shm_mq.c
+++ b/src/backend/storage/ipc/shm_mq.c
@@ -746,6 +746,15 @@ shm_mq_detach(shm_mq *mq)
}
/*
+ * Get the shm_mq from handle.
+ */
+shm_mq *
+shm_mq_get_queue(shm_mq_handle *mqh)
+{
+ return mqh->mqh_queue;
+}
+
+/*
* Write bytes into a shared message queue.
*/
static shm_mq_result
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..57014ee 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -44,9 +45,10 @@
* dummy DestReceiver functions
* ----------------
*/
-static void
+static bool
donothingReceive(TupleTableSlot *slot, DestReceiver *self)
{
+ return true;
}
static void
@@ -129,6 +131,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +167,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +210,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +255,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7c18298..89938e2 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,8 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/execParallel.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -1192,6 +1194,94 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ PopulateParamExecParams(queryDesc, parallelstmt->serialized_param_exec_vals);
+
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins like pg_stat_statements to report the total usage for
+ * statement execution.
+ */
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index bcffd85..50c717b 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -1127,7 +1127,13 @@ RunFromStore(Portal portal, ScanDirection direction, long count,
if (!ok)
break;
- (*dest->receiveSlot) (slot, dest);
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
ExecClearTuple(slot);
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 3038d7c..6d9924b 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -603,6 +603,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2542,6 +2544,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2729,6 +2741,26 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 06dfc06..32ff938 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,8 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +503,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index eec7c95..4700241 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -96,8 +96,9 @@ extern Relation heap_openrv_extended(const RangeVar *relation,
#define heap_close(r,l) relation_close(r,l)
-/* struct definition appears in relscan.h */
+/* struct definitions appear in relscan.h */
typedef struct HeapScanDescData *HeapScanDesc;
+typedef struct ParallelHeapScanDescData *ParallelHeapScanDesc;
/*
* HeapScanIsValid
@@ -121,9 +122,15 @@ extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heapgetpage(HeapScanDesc scan, BlockNumber page);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
+extern Size heap_parallelscan_estimate(Snapshot snapshot);
+extern void heap_parallelscan_initialize(ParallelHeapScanDesc target,
+ Relation relation, Snapshot snapshot);
+extern HeapScanDesc heap_beginscan_parallel(Relation, ParallelHeapScanDesc);
+
extern bool heap_fetch(Relation relation, Snapshot snapshot,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
Relation stats_relation);
diff --git a/src/include/access/printtup.h b/src/include/access/printtup.h
index 46c4148..92ec882 100644
--- a/src/include/access/printtup.h
+++ b/src/include/access/printtup.h
@@ -25,11 +25,11 @@ extern void SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist,
extern void debugStartup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void debugtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool debugtup(TupleTableSlot *slot, DestReceiver *self);
/* XXX these are really in executor/spi.c */
extern void spi_dest_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void spi_printtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool spi_printtup(TupleTableSlot *slot, DestReceiver *self);
#endif /* PRINTTUP_H */
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 1b9b299..f28e9f9 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,6 +20,15 @@
#include "access/itup.h"
#include "access/tupdesc.h"
+/* Struct for parallel scan setup */
+typedef struct ParallelHeapScanDescData
+{
+ Oid phs_relid;
+ BlockNumber phs_nblocks;
+ slock_t phs_mutex;
+ BlockNumber phs_cblock;
+ char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelHeapScanDescData;
typedef struct HeapScanDescData
{
@@ -49,6 +58,7 @@ typedef struct HeapScanDescData
BlockNumber rs_cblock; /* current block # in scan, if any */
Buffer rs_cbuf; /* current buffer in scan, if any */
/* NB: if rs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
+ ParallelHeapScanDesc rs_parallel; /* parallel scan information */
/* these fields only used in page-at-a-time mode and for bitmap scans */
int rs_cindex; /* current tuple's index in vistuples */
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
new file mode 100644
index 0000000..73006a8
--- /dev/null
+++ b/src/include/executor/execParallel.h
@@ -0,0 +1,65 @@
+/*--------------------------------------------------------------------
+ * execParallel.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/executor/execParallel.h
+ *--------------------------------------------------------------------
+ */
+#ifndef EXECPARALLEL_H
+#define EXECPARALLEL_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+#include "nodes/execnodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_PARAMS_EXEC 2
+#define PARALLEL_KEY_BUFF_USAGE 3
+#define PARALLEL_KEY_INST_OPTIONS 4
+#define PARALLEL_KEY_INST_INFO 5
+#define PARALLEL_KEY_TUPLE_QUEUE 6
+#define PARALLEL_KEY_SCAN 7
+
+extern int parallel_seqscan_degree;
+
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
+
+extern void InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+extern shm_toc *GetParallelShmToc(void);
+extern bool ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelBufferUsageAccum(Node *node);
+extern void ExecAssociateBufferStatsToDSM(BufferUsage *buf_usage,
+ ParallelStmt *parallel_stmt);
+#endif /* EXECPARALLEL_H */
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index e60ab9f..c3e4e7f 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -273,6 +273,8 @@ extern TupleDesc ExecCleanTypeFromTL(List *targetList, bool hasoid);
extern TupleDesc ExecTypeFromExprList(List *exprList);
extern void ExecTypeSetColNames(TupleDesc typeInfo, List *namesList);
extern void UpdateChangedParamSet(PlanState *node, Bitmapset *newchg);
+extern void PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals);
typedef struct TupOutputState
{
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index c9a2129..0c7847d 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..27d0b3d
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void FinishParallelSetupAndAccumStats(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..47b8f73
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node,
+ EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/nodeSubplan.h b/src/include/executor/nodeSubplan.h
index 3732ad4..21c745e 100644
--- a/src/include/executor/nodeSubplan.h
+++ b/src/include/executor/nodeSubplan.h
@@ -24,4 +24,7 @@ extern void ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent);
extern void ExecSetParamPlan(SubPlanState *node, ExprContext *econtext);
+extern List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params);
+
#endif /* NODESUBPLAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..d2ddb6e
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,35 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void TupleQueueFunnelShutdown(TupleQueueFunnel *funnel);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 0a92cc4..09ffb08 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
@@ -401,6 +403,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1050,6 +1064,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1264,6 +1283,45 @@ typedef struct SampleScanState
} SampleScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodeFuncs.h b/src/include/nodes/nodeFuncs.h
index 7b1b1d6..df00d3d 100644
--- a/src/include/nodes/nodeFuncs.h
+++ b/src/include/nodes/nodeFuncs.h
@@ -13,6 +13,7 @@
#ifndef NODEFUNCS_H
#define NODEFUNCS_H
+#include "access/parallel.h"
#include "nodes/parsenodes.h"
@@ -63,4 +64,7 @@ extern Node *query_or_expression_tree_mutator(Node *node, Node *(*mutator) (),
extern bool raw_expression_tree_walker(Node *node, bool (*walker) (),
void *context);
+extern bool planstate_tree_walker(Node *node, ParallelContext *pcxt,
+ bool (*walker) (), void *context);
+
#endif /* NODEFUNCS_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 669a0af..9eb344f 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -99,6 +101,8 @@ typedef enum NodeTag
T_ScanState,
T_SeqScanState,
T_SampleScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -223,6 +227,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..21c6f7a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -14,6 +14,8 @@
#ifndef PARAMS_H
#define PARAMS_H
+#include "nodes/pg_list.h"
+
/* To avoid including a pile of parser headers, reference ParseState thus: */
struct ParseState;
@@ -96,11 +98,47 @@ typedef struct ParamExecData
{
void *execPlan; /* should be "SubPlanState *" */
Datum value;
+ /*
+ * parameter's datatype, or 0. This is required so that
+ * datum value can be read and used for other purposes like
+ * passing it to worker backend via shared memory. This is
+ * required only for evaluation of initPlan's, however for
+ * consistency we set this for Subplan as well. We left it
+ * for other cases like CTE or RecursiveUnion cases where this
+ * structure is not used for evaluation of subplans.
+ */
+ Oid ptype;
bool isnull;
} ParamExecData;
+/*
+ * This structure is used to pass PARAM_EXEC parameters to backend
+ * workers. For each PARAM_EXEC parameter, pass this structure
+ * followed by value except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExecData
+{
+ int paramid; /* parameter id of this param */
+ Size length; /* length of parameter value */
+ Oid ptype; /* parameter's datatype, or 0 */
+ Datum value;
+ bool isnull;
+} SerializedParamExecData;
+
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
+extern Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals);
+extern void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address);
+List *
+RestoreExecParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index ff45838..9cb1853 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -294,6 +294,22 @@ typedef Scan SeqScan;
typedef Scan SampleScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index c652213..1eea6d3 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -752,6 +752,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 24003ae..a1c9f59 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,13 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +55,11 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -69,6 +79,11 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel);
+extern void cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 89c8ded..e84f925 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -34,6 +34,10 @@ extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
extern Path *create_samplescan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nworkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 3e2378a..bd8eb67 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -55,6 +55,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 52b077a..67a8582 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -133,6 +133,7 @@ extern bool query_is_distinct_for(Query *query, List *colnos, List *opids);
*/
extern Plan *set_plan_references(PlannerInfo *root, Plan *plan);
extern void fix_opfuncids(Node *node);
+extern void fix_node_funcids(Plan *node);
extern void set_opfuncid(OpExpr *opexpr);
extern void set_sa_opfuncid(ScalarArrayOpExpr *opexpr);
extern void record_plan_function_dependency(PlannerInfo *root, Oid funcid);
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..8c7ce75 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable, int num_exec_params);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/storage/shm_mq.h b/src/include/storage/shm_mq.h
index 085a8a7..74e288c 100644
--- a/src/include/storage/shm_mq.h
+++ b/src/include/storage/shm_mq.h
@@ -65,6 +65,9 @@ extern void shm_mq_set_handle(shm_mq_handle *, BackgroundWorkerHandle *);
/* Break connection. */
extern void shm_mq_detach(shm_mq *);
+/* Get the shm_mq from handle. */
+extern shm_mq *shm_mq_get_queue(shm_mq_handle *mqh);
+
/* Send or receive messages. */
extern shm_mq_result shm_mq_send(shm_mq_handle *mqh,
Size nbytes, const void *data, bool nowait);
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..91acd60 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
@@ -103,7 +104,9 @@ typedef enum
* pointers that the executor must call.
*
* Note: the receiveSlot routine must be passed a slot containing a TupleDesc
- * identical to the one given to the rStartup routine.
+ * identical to the one given to the rStartup routine. It returns bool where
+ * a "true" value means "continue processing" and a "false" value means
+ * "stop early, just as if we'd reached the end of the scan".
* ----------------
*/
typedef struct _DestReceiver DestReceiver;
@@ -111,7 +114,7 @@ typedef struct _DestReceiver DestReceiver;
struct _DestReceiver
{
/* Called for each tuple to be output: */
- void (*receiveSlot) (TupleTableSlot *slot,
+ bool (*receiveSlot) (TupleTableSlot *slot,
DestReceiver *self);
/* Per-executor-run initialization and shutdown: */
void (*rStartup) (DestReceiver *self,
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 96c5b8b..6f319c1 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -19,6 +19,7 @@
#ifndef TCOPPROT_H
#define TCOPPROT_H
+#include "executor/execParallel.h"
#include "nodes/params.h"
#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
@@ -84,5 +85,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index c0f9cb9..38b91f8 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
[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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, Jun 30, 2015 at 4:00 AM, Jeff Davis <pgsql@j-davis.com> wrote:
[Jumping in without catching up on entire thread.
No problem.
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.)
For what you are asking to change name for?
We have two nodes in patch (Funnel and PartialSeqScan). Funnel is
the name given to node because it is quite generic and can be
used in multiple ways (other than plain parallel sequiantial scan)
and other node is named as PartialSeqScan because it is used
for doing the part of sequence scan.
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]/messages/by-id/CAA4eK1JHCmN2X1LjQ4bOmLApt+btOuid5Vqqk5G6dDFV69iyHg@mail.gmail.com, though it helps when the data is cached
and there are really good benefits in terms of CPU [2]Refer slides 14-15 for the presentation in PGCon, I can repost the data here if required. https://www.pgcon.org/2015/schedule/events/785.en.html.
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?
[1]: /messages/by-id/CAA4eK1JHCmN2X1LjQ4bOmLApt+btOuid5Vqqk5G6dDFV69iyHg@mail.gmail.com
/messages/by-id/CAA4eK1JHCmN2X1LjQ4bOmLApt+btOuid5Vqqk5G6dDFV69iyHg@mail.gmail.com
[2]: Refer slides 14-15 for the presentation in PGCon, I can repost the data here if required. https://www.pgcon.org/2015/schedule/events/785.en.html
data here if required.
https://www.pgcon.org/2015/schedule/events/785.en.html
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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. DocumentationSuggestions?
A src/backend/executor/README.parallel?
Thanks,
Amit
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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 ScanOn 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jul 1, 2015 at 1:21 PM, Jeff Davis <pgsql@j-davis.com> wrote:
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.
In the initial version of patch, there was only one node parallel seqscan
node and the occurrences you are seeing are left over's, I will change
them in next patch.
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?
Yes, IIRC there was some discussion happened and I haven't used for
the reason you mentioned. It doesn't same sane to hold the pin on
page for long time (we need to retain the pin till master backend processes
that tuple).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Jul 2, 2015 at 1:47 AM, Gavin Flower <GavinFlower@archidevsys.co.nz>
wrote:
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,
please feel free to test and share the data if you have access to such
m/c's.
Attached, find the rebased version of patch.
Note - You need to first apply the assess-parallel-safety patch which you
can find at:
/messages/by-id/CAA4eK1JjsfE_dOsHTr_z1P_cBKi_X4C4X3d7Nv=VWX9fs7qdJA@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_v16.patchapplication/octet-stream; name=parallel_seqscan_v16.patchDownload
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..639451a 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -26,9 +26,9 @@
static void printtup_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-static void printtup(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_20(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
static void printtup_shutdown(DestReceiver *self);
static void printtup_destroy(DestReceiver *self);
@@ -299,7 +299,7 @@ printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
* printtup --- print a tuple in protocol 3.0
* ----------------
*/
-static void
+static bool
printtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -376,13 +376,15 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
* printtup_20 --- print a tuple in protocol 2.0
* ----------------
*/
-static void
+static bool
printtup_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -452,6 +454,8 @@ printtup_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
@@ -528,7 +532,7 @@ debugStartup(DestReceiver *self, int operation, TupleDesc typeinfo)
* debugtup - print one tuple for an interactive backend
* ----------------
*/
-void
+bool
debugtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -553,6 +557,8 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
printatt((unsigned) i + 1, typeinfo->attrs[i], value);
}
printf("\t----\n");
+
+ return true;
}
/* ----------------
@@ -564,7 +570,7 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
* This is largely same as printtup_20, except we use binary formatting.
* ----------------
*/
-static void
+static bool
printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -636,4 +642,6 @@ printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 86a2e6b..f5242a4 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -63,6 +63,7 @@
#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
+#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/datum.h"
#include "utils/inval.h"
@@ -80,9 +81,11 @@ bool synchronize_seqscans = true;
static HeapScanDesc heap_beginscan_internal(Relation relation,
Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync, bool allow_pagemode,
bool is_bitmapscan, bool is_samplescan,
bool temp_snap);
+static BlockNumber heap_parallelscan_nextpage(ParallelHeapScanDesc);
static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
TransactionId xid, CommandId cid, int options);
static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
@@ -223,7 +226,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool is_rescan)
* results for a non-MVCC snapshot, the caller must hold some higher-level
* lock that ensures the interesting tuple(s) won't change.)
*/
- scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
+ if (scan->rs_parallel != NULL)
+ scan->rs_nblocks = scan->rs_parallel->phs_nblocks;
+ else
+ scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
/*
* If the table is large relative to NBuffers, use a bulk-read access
@@ -483,7 +489,18 @@ heapgettup(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineoff = FirstOffsetNumber; /* first offnum */
scan->rs_inited = true;
@@ -506,6 +523,9 @@ heapgettup(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -658,11 +678,19 @@ heapgettup(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -760,7 +788,18 @@ heapgettup_pagemode(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ if (page >= scan->rs_nblocks)
+ {
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineindex = 0;
scan->rs_inited = true;
@@ -780,6 +819,9 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -921,11 +963,19 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ page = heap_parallelscan_nextpage(scan->rs_parallel);
+ finished = (page >= scan->rs_nblocks);
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -1321,7 +1371,7 @@ HeapScanDesc
heap_beginscan(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, false);
}
@@ -1331,7 +1381,7 @@ heap_beginscan_catalog(Relation relation, int nkeys, ScanKey key)
Oid relid = RelationGetRelid(relation);
Snapshot snapshot = RegisterSnapshot(GetCatalogSnapshot(relid));
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, true);
}
@@ -1340,7 +1390,7 @@ heap_beginscan_strat(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, true,
false, false, false);
}
@@ -1349,7 +1399,7 @@ HeapScanDesc
heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
false, false, true, true, false, false);
}
@@ -1358,7 +1408,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_pagemode)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, false, allow_pagemode,
false, true, false);
}
@@ -1366,6 +1416,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
static HeapScanDesc
heap_beginscan_internal(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat, bool allow_sync, bool allow_pagemode,
bool is_bitmapscan, bool is_samplescan, bool temp_snap)
{
@@ -1394,6 +1445,7 @@ heap_beginscan_internal(Relation relation, Snapshot snapshot,
scan->rs_allow_strat = allow_strat;
scan->rs_allow_sync = allow_sync;
scan->rs_temp_snap = temp_snap;
+ scan->rs_parallel = parallel_scan;
/*
* we can use page-at-a-time mode if it's an MVCC-safe snapshot
@@ -1487,6 +1539,94 @@ heap_endscan(HeapScanDesc scan)
}
/* ----------------
+ * heap_parallelscan_estimate - estimate storage for ParallelHeapScanDesc
+ *
+ * Sadly, this doesn't reduce to a constant, because the size required
+ * to serialize the snapshot can vary.
+ * ----------------
+ */
+Size
+heap_parallelscan_estimate(Snapshot snapshot)
+{
+ return add_size(offsetof(ParallelHeapScanDescData, phs_snapshot_data),
+ EstimateSnapshotSpace(snapshot));
+}
+
+/* ----------------
+ * heap_parallelscan_initialize - initialize ParallelHeapScanDesc
+ *
+ * Must allow as many bytes of shared memory as returned by
+ * heap_parallelscan_estimate. Call this just once in the leader
+ * process; then, individual workers attach via heap_beginscan_parallel.
+ * ----------------
+ */
+void
+heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
+ Snapshot snapshot)
+{
+ target->phs_relid = RelationGetRelid(relation);
+ target->phs_nblocks = RelationGetNumberOfBlocks(relation);
+ SpinLockInit(&target->phs_mutex);
+ target->phs_cblock = 0;
+ SerializeSnapshot(snapshot, target->phs_snapshot_data);
+}
+/* ----------------
+ * heap_parallelscan_nextpage - get the next page to scan
+ *
+ * A return value larger than the number of blocks to be scanned
+ * indicates end of scan. Note, however, that other backends could still
+ * be scanning if they grabbed a page to scan and aren't done with it yet.
+ * ----------------
+ */
+static BlockNumber
+heap_parallelscan_nextpage(ParallelHeapScanDesc parallel_scan)
+{
+ BlockNumber page = InvalidBlockNumber;
+
+ /* we treat InvalidBlockNumber specially here to avoid overflow */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock != InvalidBlockNumber)
+ page = parallel_scan->phs_cblock++;
+ SpinLockRelease(¶llel_scan->phs_mutex);
+
+ return page;
+}
+
+/* ----------------
+ * heap_beginscan_parallel - join a parallel scan
+ *
+ * Caller must hold a suitable lock on the correct relation.
+ * ----------------
+ */
+HeapScanDesc
+heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
+{
+ Snapshot snapshot;
+
+ Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
+ snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
+ RegisterSnapshot(snapshot);
+
+ return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
+ true, true, true, false, false, true);
+}
+
+/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8904676..47063c7 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -4395,7 +4395,7 @@ copy_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* copy_dest_receive --- receive one tuple
*/
-static void
+static bool
copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_copy *myState = (DR_copy *) self;
@@ -4407,6 +4407,8 @@ copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
/* And send the data */
CopyOneRowTo(cstate, InvalidOid, slot->tts_values, slot->tts_isnull);
myState->processed++;
+
+ return true;
}
/*
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index 41183f6..418b0f6 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -62,7 +62,7 @@ typedef struct
static ObjectAddress CreateAsReladdr = {InvalidOid, InvalidOid, 0};
static void intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void intorel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool intorel_receive(TupleTableSlot *slot, DestReceiver *self);
static void intorel_shutdown(DestReceiver *self);
static void intorel_destroy(DestReceiver *self);
@@ -482,7 +482,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* intorel_receive --- receive one tuple
*/
-static void
+static bool
intorel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_intorel *myState = (DR_intorel *) self;
@@ -507,6 +507,8 @@ intorel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 2b930f7..9d6b663 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -20,6 +20,7 @@
#include "commands/defrem.h"
#include "commands/prepare.h"
#include "executor/hashjoin.h"
+#include "executor/nodeFunnel.h"
#include "foreign/fdwapi.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
@@ -730,6 +731,8 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -935,6 +938,12 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SeqScan:
pname = sname = "Seq Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1101,6 +1110,8 @@ ExplainNode(PlanState *planstate, List *ancestors,
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1248,6 +1259,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. Though we already accumulate this
+ * information when last tuple is fetched from Funnel node, this
+ * is to cover cases when we don't fetch all tuples from a node
+ * such as for Limit node.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ FinishParallelSetupAndAccumStats((FunnelState *)planstate);
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1364,6 +1385,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_tidbitmap_info((BitmapHeapScanState *) planstate, es);
break;
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -1374,6 +1396,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2366,6 +2396,8 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 5492e59..750a59c 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -56,7 +56,7 @@ typedef struct
static int matview_maintenance_depth = 0;
static void transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
static void transientrel_shutdown(DestReceiver *self);
static void transientrel_destroy(DestReceiver *self);
static void refresh_matview_datafill(DestReceiver *dest, Query *query,
@@ -422,7 +422,7 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* transientrel_receive --- receive one tuple
*/
-static void
+static bool
transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_transientrel *myState = (DR_transientrel *) self;
@@ -441,6 +441,8 @@ transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index 08cba6f..be1f47e 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -13,17 +13,17 @@ top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = execAmi.o execCurrent.o execGrouping.o execIndexing.o execJunk.o \
- execMain.o execProcnode.o execQual.o execScan.o execTuples.o \
+ execMain.o execParallel.o execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
- nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeNestloop.o nodeFunctionscan.o nodePartialSeqscan.o nodeRecursiveunion.o \
+ nodeResult.o nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 04073d3..233e584 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -37,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -160,6 +162,14 @@ ExecReScan(PlanState *node)
ExecReScanSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -463,6 +473,10 @@ ExecSupportsBackwardScan(Plan *node)
case T_CteScan:
return TargetListSupportsBackwardScan(node->targetlist);
+ case T_Funnel:
+ case T_PartialSeqScan:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index bcd287f..7a44462 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -262,6 +262,8 @@ search_plan_tree(PlanState *node, Oid table_oid)
*/
case T_SeqScanState:
case T_SampleScanState:
+ case T_PartialSeqScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 2e23cc7..e252727 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -45,9 +45,11 @@
#include "commands/matview.h"
#include "commands/trigger.h"
#include "executor/execdebug.h"
+#include "executor/execParallel.h"
#include "foreign/fdwapi.h"
#include "mb/pg_wchar.h"
#include "miscadmin.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "parser/parsetree.h"
#include "storage/bufmgr.h"
@@ -323,6 +325,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
@@ -354,7 +359,15 @@ standard_ExecutorRun(QueryDesc *queryDesc,
(*dest->rShutdown) (dest);
if (queryDesc->totaltime)
+ {
+ /*
+ * Accumulate the stats by parallel workers before stopping the
+ * node.
+ */
+ (void) planstate_tree_walker((Node*) queryDesc->planstate,
+ NULL, ExecParallelBufferUsageAccum, 0);
InstrStopNode(queryDesc->totaltime, estate->es_processed);
+ }
MemoryContextSwitchTo(oldcontext);
}
@@ -1581,7 +1594,15 @@ ExecutePlan(EState *estate,
* practice, this is probably always the case at this point.)
*/
if (sendTuples)
- (*dest->receiveSlot) (slot, dest);
+ {
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
+ }
/*
* Count tuples processed, if this is a SELECT. (For other operation
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
new file mode 100644
index 0000000..33e83fe
--- /dev/null
+++ b/src/backend/executor/execParallel.c
@@ -0,0 +1,592 @@
+/*-------------------------------------------------------------------------
+ *
+ * execParallel.c
+ * Support routines for setting up backend workers for parallel execution.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/execParallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodePartialSeqscan.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+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);
+static void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+static shm_toc *parallel_shm_toc = NULL;
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of
+ * bind parameters, PARAM_EXEC parameters and instrumentation
+ * information that need to be retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ List *serialized_param_exec_vals,
+ int instOptions, Size *params_size,
+ Size *params_exec_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ *params_exec_size = EstimateExecParametersSpace(serialized_param_exec_vals);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_exec_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 4);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters, PARAM_EXEC parameters and instrumentation
+ * information required for parallel execution.
+ */
+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)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Store PARAM_EXEC parameters list in dynamic shared memory. This is
+ * used for evaluation plan->initPlan params.
+ */
+ paramsexecdata = shm_toc_allocate(pcxt->toc, params_exec_size);
+ SerializeExecParams(serialized_param_exec_vals, params_exec_size, paramsexecdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS_EXEC, paramsexecdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePlannedStmtSpace
+ *
+ * Estimate the amount of space required to record information of
+ * planned statement and parallel node specific information that need
+ * to be copied to parallel workers.
+ */
+void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for planned statement. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ /* keys for planned statement information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelEstimate,
+ pscan_size);
+}
+
+/*
+ * StorePlannedStmt
+ *
+ * Sets up the planned statement and node specific information.
+ */
+void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+
+ /* Store planned statement in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelInitializeDSM,
+ &pscan_size);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of
+ * tuple queues that need to be established between parallel workers
+ * and master backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for parallel seq. scan specific contents. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queue's for backend worker's to
+ * return tuples to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. (Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.)
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * ExecParallelEstimate
+ *
+ * Estimate the amount of space required to record information of
+ * parallel node that need to be copied to parallel workers.
+ */
+bool
+ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ if (node == NULL)
+ return false;
+
+ switch (nodeTag(node))
+ {
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelEstimate, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* key for paratial scan information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ return true;
+ }
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * ExecParallelInitializeDSM
+ *
+ * Store the information of parallel node in dsm.
+ */
+bool
+ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ ParallelHeapScanDesc pscan;
+
+ if (node == NULL)
+ return false;
+
+ switch (nodeTag(node))
+ {
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelInitializeDSM, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, *pscan_size);
+ heap_parallelscan_initialize(pscan, ((PartialSeqScanState*)node)->ss.ss_currentRelation, estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+ return true;
+ }
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to
+ * perform execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, params_exec_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt((PartialSeqScan *)planstate->plan,
+ estate->es_range_table,
+ estate->es_plannedstmt->nParamExec);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePlannedStmtSpace(pcxt, planstate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument, ¶ms_size,
+ ¶ms_exec_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePlannedStmt(pcxt, planstate, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument,
+ params_size,
+ params_exec_size,
+ inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the bind parameters, PARAM_EXEC parameters and
+ * instrumentation information required to perform parallel
+ * operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ if (params)
+ {
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ *params = RestoreBoundParams(paramsdata);
+ }
+
+ if (serialized_param_exec_vals)
+ {
+ paramsexecdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS_EXEC);
+ *serialized_param_exec_vals = RestoreExecParams(paramsexecdata);
+ }
+
+ if (inst_options)
+ {
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+ }
+
+ if (buffer_usage)
+ {
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+ }
+}
+
+/*
+ * ExecParallelGetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the planned statement required to perform
+ * parallel operation.
+ */
+void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_node_funcids((*plannedstmt)->planTree);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment
+ * and get the tuple queue information for a particular worker,
+ * attach to the queue and redirect all futher responses from
+ * worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * GetParallelShmToc
+ */
+shm_toc *
+GetParallelShmToc(void)
+{
+ return parallel_shm_toc;
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information
+ * to parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ ExecParallelGetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &serialized_param_exec_vals,
+ &inst_options, &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->serialized_param_exec_vals = serialized_param_exec_vals;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->responseq = responseq;
+
+ parallel_shm_toc = toc;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from
+ * shared memory message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
+
+/*
+ * ExecParallelBufferUsageAccum
+ *
+ * Recursively accumulate the stats for all the funnel nodes
+ * in a plan state tree.
+ */
+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;
+}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 03c2feb..e24a439 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,8 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -196,6 +198,16 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -416,6 +428,14 @@ ExecProcNode(PlanState *node)
result = ExecSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -658,6 +678,14 @@ ExecEndNode(PlanState *node)
ExecEndSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index a05d8b1..d5619bd 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -1313,7 +1313,7 @@ do_tup_output(TupOutputState *tstate, Datum *values, bool *isnull)
ExecStoreVirtualTuple(slot);
/* send the tuple to the receiver */
- (*tstate->dest->receiveSlot) (slot, tstate->dest);
+ (void) (*tstate->dest->receiveSlot) (slot, tstate->dest);
/* clean up */
ExecClearTuple(slot);
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 3c611b9..27ca0fa 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -976,3 +976,28 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
MemoryContextSwitchTo(oldcontext);
}
+
+/*
+ * Populate the values of PARAM_EXEC parameters.
+ *
+ * This is used by worker backends to fill in the values
+ * of PARAM_EXEC parameters after fetching the same from
+ * dynamic shared memory. This needs to be called before
+ * ExecutorRun.
+ */
+void
+PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals)
+{
+ ListCell *lparam;
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].value =
+ param_val->value;
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].isnull =
+ param_val->isnull;
+ }
+}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 812a610..863bd64 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -167,7 +167,7 @@ static Datum postquel_get_single_result(TupleTableSlot *slot,
static void sql_exec_error_callback(void *arg);
static void ShutdownSQLFunction(Datum arg);
static void sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
static void sqlfunction_shutdown(DestReceiver *self);
static void sqlfunction_destroy(DestReceiver *self);
@@ -1903,7 +1903,7 @@ sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* sqlfunction_receive --- receive one tuple
*/
-static void
+static bool
sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_sqlfunction *myState = (DR_sqlfunction *) self;
@@ -1913,6 +1913,8 @@ sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
/* Store the filtered tuple into the tuplestore */
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..283a136 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,30 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used
+ * to aggregate the information of worker backends. We only
+ * need to sum the buffer usage and tuple count statistics as
+ * for other timing related statistics it is sufficient to
+ * have the master backend's information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +167,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..3c42f21
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,436 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel rescans a relation
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodeSubplan.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+static void ExecAccumulateInstInfo(FunnelState *node);
+static void ExecAccumulateBufUsageInfo(FunnelState *node);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ ExprContext *econtext = node->ss.ps.ps_ExprContext;
+ bool any_worker_launched = false;
+ List *serialized_param_exec;
+
+ /*
+ * 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);
+
+ /* Initialize the workers required to execute funnel node. */
+ InitializeParallelWorkers(node->ss.ps.lefttree,
+ serialized_param_exec,
+ estate,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * If there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ if (TupIsNull(slot))
+ {
+
+ /*
+ * Destroy the parallel context once we complete fetching all
+ * the tuples, this will ensure that if in the same statement
+ * we need to have Funnel node for multiple parts of statement,
+ * it won't accumulate lot of dsm's and workers can be made
+ * available to use by other parts of statement.
+ */
+ FinishParallelSetupAndAccumStats(node);
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ FinishParallelSetupAndAccumStats(node);
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * FinishParallelSetupAndAccumStats
+ *
+ * Destroy the setup for parallel workers. Collect all the
+ * stats after workers are stopped, else some work done by
+ * workers won't be accounted.
+ * ----------------------------------------------------------------
+ */
+void
+FinishParallelSetupAndAccumStats(FunnelState *node)
+{
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ {
+ TupleQueueFunnelShutdown(node->funnel);
+ WaitForParallelWorkersToFinish(node->pcxt);
+ }
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+ node->funnel = NULL;
+
+ /*
+ * Aggregate the buffer usage stats from all workers. This is
+ * required by external modules like pg_stat_statements.
+ */
+ ExecAccumulateBufUsageInfo(node);
+
+ /*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. This has to be done before we
+ * destroy the parallel context.
+ */
+ if (node->ss.ps.state->es_instrument)
+ ExecAccumulateInstInfo(node);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateInstInfo
+ *
+ * Accumulate instrumentation information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateInstInfo(FunnelState *node)
+{
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers;
+ char *inst_info_workers;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ inst_info_workers = node->inst_options_space;
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(node->ss.ps.instrument, instrument_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateBufUsageInfo
+ *
+ * Accumulate buffer usage information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateBufUsageInfo(FunnelState *node)
+{
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Rescans a relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * 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);
+
+ ExecReScan(node->ss.ps.lefttree);
+}
diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c
index e66bcda..c447062 100644
--- a/src/backend/executor/nodeNestloop.c
+++ b/src/backend/executor/nodeNestloop.c
@@ -144,6 +144,7 @@ ExecNestLoop(NestLoopState *node)
{
NestLoopParam *nlp = (NestLoopParam *) lfirst(lc);
int paramno = nlp->paramno;
+ TupleDesc tdesc = outerTupleSlot->tts_tupleDescriptor;
ParamExecData *prm;
prm = &(econtext->ecxt_param_exec_vals[paramno]);
@@ -154,6 +155,7 @@ ExecNestLoop(NestLoopState *node)
prm->value = slot_getattr(outerTupleSlot,
nlp->paramval->varattno,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[nlp->paramval->varattno-1]->atttypid;
/* Flag parameter value as changed */
innerPlan->chgParam = bms_add_member(innerPlan->chgParam,
paramno);
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..09b7e07
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,308 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for parallel sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation.
+ * PartialSeqNext retrieve next tuple from either heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodePartialSeqscan.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ shm_toc *toc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We set 'toc' (place to lookup parallel scan
+ * descriptor) as retrievied by attaching to dsm for parallel workers
+ * whereas master backend stores it directly in partial scan state node
+ * after initializing workers.
+ */
+ toc = GetParallelShmToc();
+ if (toc)
+ node->ss.ps.toc = toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ node->scan_initialized = false;
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c
index 8d3dde0..b348bfd 100644
--- a/src/backend/executor/nodeResult.c
+++ b/src/backend/executor/nodeResult.c
@@ -75,6 +75,13 @@ ExecResult(ResultState *node)
econtext = node->ps.ps_ExprContext;
/*
+ * Result node can be added as a gating node on top of PartialSeqScan
+ * node, so need to percolate toc information to outer node.
+ */
+ if (node->ps.toc)
+ outerPlanState(node)->toc = node->ps.toc;
+
+ /*
* check constant qualifications like (2 > 1), if not already done
*/
if (node->rs_checkqual)
diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c
index 9eb4d63..6afd55a 100644
--- a/src/backend/executor/nodeSubplan.c
+++ b/src/backend/executor/nodeSubplan.c
@@ -30,11 +30,14 @@
#include <math.h>
#include "access/htup_details.h"
+#include "catalog/pg_type.h"
#include "executor/executor.h"
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "utils/array.h"
+#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
@@ -281,12 +284,14 @@ ExecScanSubPlan(SubPlanState *node,
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -399,6 +404,7 @@ ExecScanSubPlan(SubPlanState *node,
prmdata = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col, &(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
@@ -551,6 +557,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
!TupIsNull(slot);
slot = ExecProcNode(planstate))
{
+ TupleDesc tdesc = slot->tts_tupleDescriptor;
int col = 1;
ListCell *plst;
bool isnew;
@@ -568,6 +575,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col,
&(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
slot = ExecProject(node->projRight, NULL);
@@ -954,6 +962,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
ListCell *l;
bool found = false;
ArrayBuildStateAny *astate = NULL;
+ Oid ptype;
if (subLinkType == ANY_SUBLINK ||
subLinkType == ALL_SUBLINK)
@@ -961,6 +970,8 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (subLinkType == CTE_SUBLINK)
elog(ERROR, "CTE subplans should not be executed via ExecSetParamPlan");
+ ptype = exprType((Node *) node->xprstate.expr);
+
/* Initialize ArrayBuildStateAny in caller's context, if needed */
if (subLinkType == ARRAY_SUBLINK)
astate = initArrayResultAny(subplan->firstColType,
@@ -983,12 +994,14 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -1011,6 +1024,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(true);
+ prm->ptype = ptype;
prm->isnull = false;
found = true;
break;
@@ -1062,6 +1076,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = heap_getattr(node->curTuple, i, tdesc,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[i-1]->atttypid;
i++;
}
}
@@ -1084,6 +1099,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
true);
prm->execPlan = NULL;
prm->value = node->curArray;
+ prm->ptype = ptype;
prm->isnull = false;
}
else if (!found)
@@ -1096,6 +1112,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(false);
+ prm->ptype = ptype;
prm->isnull = false;
}
else
@@ -1108,6 +1125,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = (Datum) 0;
+ prm->ptype = VOIDOID;
prm->isnull = true;
}
}
@@ -1238,3 +1256,47 @@ ExecAlternativeSubPlan(AlternativeSubPlanState *node,
isNull,
isDone);
}
+
+/*
+ * ExecAndFormSerializeParamExec
+ *
+ * Execute the subplan stored in PARAM_EXEC param if it is not executed
+ * till now and form the serialized structure required for passing to
+ * worker backend.
+ */
+List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params)
+{
+ List *lparam = NIL;
+ SerializedParamExecData *sparamdata;
+ ParamExecData *prm;
+ int paramid;
+
+ paramid = -1;
+ while ((paramid = bms_next_member(params, paramid)) >= 0)
+ {
+ /*
+ * PARAM_EXEC params (internal executor parameters) are stored in the
+ * ecxt_param_exec_vals array, and can be accessed by array index.
+ */
+ sparamdata = palloc0(sizeof(SerializedParamExecData));
+
+ prm = &(econtext->ecxt_param_exec_vals[paramid]);
+ if (prm->execPlan != NULL)
+ {
+ /* Parameter not evaluated yet, so go do it */
+ ExecSetParamPlan(prm->execPlan, econtext);
+ /* ExecSetParamPlan should have processed this param... */
+ Assert(prm->execPlan == NULL);
+ }
+
+ sparamdata->paramid = paramid;
+ sparamdata->ptype = prm->ptype;
+ sparamdata->value = prm->value;
+ sparamdata->isnull = prm->isnull;
+
+ lparam = lappend(lparam, sparamdata);
+ }
+
+ return lparam;
+}
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index d544ad9..d8ca074 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -1774,7 +1774,7 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
* store tuple retrieved by Executor into SPITupleTable
* of current SPI procedure
*/
-void
+bool
spi_printtup(TupleTableSlot *slot, DestReceiver *self)
{
SPITupleTable *tuptable;
@@ -1808,6 +1808,8 @@ spi_printtup(TupleTableSlot *slot, DestReceiver *self)
(tuptable->free)--;
MemoryContextSwitchTo(oldcxt);
+
+ return true;
}
/*
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..39acda7
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,304 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static bool
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result == SHM_MQ_DETACHED)
+ return false;
+ else if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+
+ return true;
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Detach all tuple queues that belong to funnel.
+ */
+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);
+ }
+ }
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/executor/tstoreReceiver.c b/src/backend/executor/tstoreReceiver.c
index c1fdeb7..b0862ae 100644
--- a/src/backend/executor/tstoreReceiver.c
+++ b/src/backend/executor/tstoreReceiver.c
@@ -37,8 +37,8 @@ typedef struct
} TStoreState;
-static void tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
-static void tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
/*
@@ -90,19 +90,21 @@ tstoreStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the easy case where we don't have to detoast.
*/
-static void
+static bool
tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the case where we have to detoast any toasted values.
*/
-static void
+static bool
tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
@@ -152,6 +154,8 @@ tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
/* And release any temporary detoasted values */
for (i = 0; i < nfree; i++)
pfree(DatumGetPointer(myState->tofree[i]));
+
+ return true;
}
/*
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 5bfb7c0..578b39a 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -361,6 +361,43 @@ _copySeqScan(const SeqScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4226,6 +4263,12 @@ copyObject(const void *from)
case T_SeqScan:
retval = _copySeqScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index b1e3e6e..e610d9d 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -3399,3 +3399,25 @@ raw_expression_tree_walker(Node *node,
}
return false;
}
+
+/*
+ * planstate_tree_walker
+ *
+ * This routine will invoke walker on the node passed. This is a useful
+ * way of starting the recursion when the walker's normal change of state
+ * is not appropriate for the outermost PlanState node.
+ */
+bool
+planstate_tree_walker(Node *node,
+ ParallelContext *pcxt,
+ bool (*walker) (),
+ void *context)
+{
+ if (node == NULL)
+ return false;
+
+ /* Guard against stack overflow due to overly complex plan */
+ check_stack_depth();
+
+ return walker(node, pcxt, context);
+}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 482354c..f8dee8e 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -446,6 +446,24 @@ _outSeqScan(StringInfo str, const SeqScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -3005,6 +3023,12 @@ _outNode(StringInfo str, const void *obj)
case T_SeqScan:
_outSeqScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..0050195 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, same as in original Param */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,355 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
+
+/*
+ * Estimate the amount of space required to serialize the PARAM_EXEC
+ * parameters.
+ */
+Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals)
+{
+ Size size;
+ ListCell *lparam;
+
+ /*
+ * Add space required for saving number of PARAM_EXEC parameters
+ * that needs to be serialized.
+ */
+ size = sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size length;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ length = sizeof(SerializedParamExecData);
+
+ get_typlenbyval(param_val->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || param_val->isnull))
+ {
+ length += datumGetSize(param_val->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the PARAM_EXEC parameters into the memory, beginning at
+ * start_address. maxsize should be at least as large as the value
+ * returned by EstimateExecParametersSpace.
+ */
+void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address)
+{
+ char *curptr;
+ SerializedParamExecData *retval;
+ ListCell *lparam;
+
+ /*
+ * First, we store the number of PARAM_EXEC parameters that needs to
+ * be serialized.
+ */
+ if (serialized_param_exec_vals)
+ * (int *) start_address = list_length(serialized_param_exec_vals);
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+
+ curptr = start_address + sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ retval = (SerializedParamExecData*) curptr;
+
+ retval->paramid = param_val->paramid;
+ retval->value = param_val->value;
+ retval->isnull = param_val->isnull;
+ retval->ptype = param_val->ptype;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(retval->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(retval->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(retval->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreExecParams
+ * Restore PARAM_EXEC parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+List *
+RestoreExecParams(char *start_address)
+{
+ List *lparamexecvals = NIL;
+ //Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExecData *nprm;
+ SerializedParamExecData *outparam;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExecData *) curptr;
+
+ outparam = palloc0(sizeof(SerializedParamExecData));
+
+ /* copy the parameter info */
+ outparam->isnull = nprm->isnull;
+ outparam->value = nprm->value;
+ outparam->paramid = nprm->paramid;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ outparam->value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+
+ lparamexecvals = lappend(lparamexecvals, outparam);
+ }
+
+ return lparamexecvals;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index f5a40fb..e3357cc 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -29,6 +29,7 @@
#include <math.h>
#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
#include "nodes/readfuncs.h"
@@ -1391,6 +1392,124 @@ _readRangeTblFunction(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlan
+ */
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+/*
+ * _readScan
+ */
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
+
+/*
+ * _readResult
+ */
+static Result *
+_readResult(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(Result);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_NODE_FIELD(resconstantqual);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1532,6 +1651,14 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
+ else if (MATCH("RESULT", 6))
+ return_value = _readResult();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 0b83189..fa6bbdf 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -471,6 +471,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 0d302f6..8b6b46c 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,8 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -101,11 +103,15 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -287,6 +293,86 @@ cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel)
}
/*
+ * cost_patialseqscan
+ * Determines and returns the cost of scanning a relation partially.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ * 'nworkers' are the number of workers among which the work will be
+ * distributed
+ */
+void
+cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ cost_seqscan(path, root, baserel, param_info);
+
+ startup_cost = path->startup_cost;
+
+ run_cost = path->total_cost - startup_cost;
+
+ /*
+ * Account for small cost for communication related to scan
+ * via the ParallelHeapScanDesc.
+ */
+ run_cost += 0.01;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * partial sequiantial scan.
+ */
+ run_cost = run_cost / (nworkers + 1);
+
+ path->startup_cost = startup_cost;
+ path->total_cost = startup_cost + run_cost;
+}
+
+/*
+ * cost_funnel
+ * Determines and returns the cost of funnel path.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..bc71737
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,89 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine which conditions are usable for scanning
+ * a given relation, and create ParallelPaths accordingly.
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support parallel sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel)
+{
+ int num_parallel_workers = 0;
+ int estimated_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set
+ * parallel_seqscan_degree to value greater than 0
+ * and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ /*
+ * There should be atleast thousand pages to scan for each worker.
+ * This number is somewhat arbitratry, however we don't want to
+ * spawn workers to scan smaller relations as that will be costly.
+ */
+ estimated_parallel_workers = rel->pages / 1000;
+
+ if (estimated_parallel_workers <= 0)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ if (parallel_seqscan_degree <= estimated_parallel_workers)
+ num_parallel_workers = parallel_seqscan_degree;
+ else
+ num_parallel_workers = estimated_parallel_workers;
+
+ /* Create the partial scan path which each worker needs to execute. */
+ subpath = create_partialseqscan_path(root, rel, false,
+ num_parallel_workers);
+
+ /* Create the parallel scan path which master needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index dc2dcbf..1e2b3ba 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -60,6 +60,10 @@ static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
static SampleScan *create_samplescan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
+static Scan *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -103,6 +107,11 @@ static void copy_path_costsize(Plan *dest, Path *src);
static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
static SampleScan *make_samplescan(List *qptlist, List *qpqual, Index scanrelid);
+static PartialSeqScan *make_partialseqscan(List *qptlist, List *qpqual,
+ Index scanrelid);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -233,6 +242,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -272,6 +282,10 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
plan = create_unique_plan(root,
(UniquePath *) best_path);
break;
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path);
+ break;
default:
elog(ERROR, "unrecognized node type: %d",
(int) best_path->pathtype);
@@ -355,6 +369,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -559,6 +580,8 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1186,6 +1209,107 @@ create_samplescan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path'.
+ */
+static Scan *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path)
+{
+ Scan *scan_plan;
+ Plan *subplan;
+ List *tlist;
+ RelOptInfo *rel = best_path->path.parent;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /*
+ * For table scans, rather than using the relation targetlist (which is
+ * only those Vars actually needed by the query), we prefer to generate a
+ * tlist containing all Vars in order. This will allow the executor to
+ * optimize away projection of the table tuples, if possible. (Note that
+ * planner.c may replace the tlist we generate here, forcing projection to
+ * occur.)
+ */
+ if (use_physical_tlist(root, rel))
+ {
+ tlist = build_physical_tlist(root, rel);
+ /* if fail because of dropped cols, use regular method */
+ if (tlist == NIL)
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+ else
+ {
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ scan_plan = (Scan *) make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&scan_plan->plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return scan_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3452,6 +3576,45 @@ make_samplescan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 7065e39..641b05f 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -293,6 +293,52 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable,
+ int num_exec_params)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, partialscan->plan.targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = (Plan*) partialscan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = num_exec_params;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 820f69d..90f4dfb 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -440,6 +440,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -461,6 +462,26 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_list(root, splan->plan.qual, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ /*
+ * target list for partial sequence scan (leftree of funnel plan)
+ * should be same as for funnel scan as both nodes need to produce
+ * same projection. We don't want to do this assignment after
+ * fixing references as that will be done separately for partial
+ * sequence scan node.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
@@ -2259,6 +2280,45 @@ fix_opfuncids_walker(Node *node, void *context)
}
/*
+ * fix_node_funcids
+ * Set the opfuncid (procedure OID) in an OpExpr node,
+ * for plan tree.
+ *
+ * We need it mainly to fix the opfuncid in nodes of plantree
+ * after reading the planned statement by worker backend.
+ * Currently the support of nodes that could be executed by
+ * worker backend are limited, so we can enhance this API based
+ * on it's usage in future.
+ */
+void
+fix_node_funcids(Plan *node)
+{
+ /*
+ * do nothing when we get to the end of a leaf on tree.
+ */
+ if (node == NULL)
+ return;
+
+ fix_opfuncids((Node*) node->qual);
+ fix_opfuncids((Node*) node->targetlist);
+
+ switch (nodeTag(node))
+ {
+ case T_Result:
+ fix_opfuncids((Node*) (((Result *)node)->resconstantqual));
+ break;
+ case T_PartialSeqScan:
+ break;
+ default:
+ elog(ERROR, "unrecognized node type: %d", (int) nodeTag(node));
+ break;
+ }
+
+ fix_node_funcids(node->lefttree);
+ fix_node_funcids(node->righttree);
+}
+
+/*
* set_opfuncid
* Set the opfuncid (procedure OID) in an OpExpr node,
* if it hasn't been set already.
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 4708b87..92e9326 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2217,6 +2217,8 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
+ case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index f7f33bb..81ca06c 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -732,6 +732,54 @@ create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->pathkeys = NIL; /* partialseqscan has unordered result */
+
+ cost_patialseqscan(pathnode, root, rel, pathnode->param_info, nworkers);
+
+ return pathnode;
+}
+
+/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel,
+ Path* subpath, int nworkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ false);
+ pathnode->path.pathkeys = NIL; /* seqscan has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nworkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/Makefile b/src/backend/postmaster/Makefile
index 71c2321..4aec92a 100644
--- a/src/backend/postmaster/Makefile
+++ b/src/backend/postmaster/Makefile
@@ -13,6 +13,7 @@ top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = autovacuum.o bgworker.o bgwriter.o checkpointer.o fork_process.o \
- pgarch.o pgstat.o postmaster.o startup.o syslogger.o walwriter.o
+ pgarch.o pgstat.o postmaster.o startup.o syslogger.o \
+ walwriter.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index df8037b..8729bef 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -848,6 +849,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/storage/ipc/shm_mq.c b/src/backend/storage/ipc/shm_mq.c
index 126cb07..e482269 100644
--- a/src/backend/storage/ipc/shm_mq.c
+++ b/src/backend/storage/ipc/shm_mq.c
@@ -746,6 +746,15 @@ shm_mq_detach(shm_mq *mq)
}
/*
+ * Get the shm_mq from handle.
+ */
+shm_mq *
+shm_mq_get_queue(shm_mq_handle *mqh)
+{
+ return mqh->mqh_queue;
+}
+
+/*
* Write bytes into a shared message queue.
*/
static shm_mq_result
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..57014ee 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -44,9 +45,10 @@
* dummy DestReceiver functions
* ----------------
*/
-static void
+static bool
donothingReceive(TupleTableSlot *slot, DestReceiver *self)
{
+ return true;
}
static void
@@ -129,6 +131,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +167,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +210,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +255,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7598318..f1542a0 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,8 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/execParallel.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -1192,6 +1194,94 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ PopulateParamExecParams(queryDesc, parallelstmt->serialized_param_exec_vals);
+
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins like pg_stat_statements to report the total usage for
+ * statement execution.
+ */
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..f2fb638 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -1121,7 +1121,13 @@ RunFromStore(Portal portal, ScanDirection direction, long count,
if (!ok)
break;
- (*dest->receiveSlot) (slot, dest);
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
ExecClearTuple(slot);
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 595a609..c4e9531 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -607,6 +607,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2545,6 +2547,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2732,6 +2744,26 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index 06dfc06..32ff938 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -291,6 +291,8 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -501,6 +503,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 31139cb..d56e839 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -96,8 +96,9 @@ extern Relation heap_openrv_extended(const RangeVar *relation,
#define heap_close(r,l) relation_close(r,l)
-/* struct definition appears in relscan.h */
+/* struct definitions appear in relscan.h */
typedef struct HeapScanDescData *HeapScanDesc;
+typedef struct ParallelHeapScanDescData *ParallelHeapScanDesc;
/*
* HeapScanIsValid
@@ -121,9 +122,15 @@ extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heapgetpage(HeapScanDesc scan, BlockNumber page);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
+extern Size heap_parallelscan_estimate(Snapshot snapshot);
+extern void heap_parallelscan_initialize(ParallelHeapScanDesc target,
+ Relation relation, Snapshot snapshot);
+extern HeapScanDesc heap_beginscan_parallel(Relation, ParallelHeapScanDesc);
+
extern bool heap_fetch(Relation relation, Snapshot snapshot,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
Relation stats_relation);
diff --git a/src/include/access/printtup.h b/src/include/access/printtup.h
index 46c4148..92ec882 100644
--- a/src/include/access/printtup.h
+++ b/src/include/access/printtup.h
@@ -25,11 +25,11 @@ extern void SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist,
extern void debugStartup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void debugtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool debugtup(TupleTableSlot *slot, DestReceiver *self);
/* XXX these are really in executor/spi.c */
extern void spi_dest_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void spi_printtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool spi_printtup(TupleTableSlot *slot, DestReceiver *self);
#endif /* PRINTTUP_H */
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index f2482e9..90af7e1 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,6 +20,15 @@
#include "access/itup.h"
#include "access/tupdesc.h"
+/* Struct for parallel scan setup */
+typedef struct ParallelHeapScanDescData
+{
+ Oid phs_relid;
+ BlockNumber phs_nblocks;
+ slock_t phs_mutex;
+ BlockNumber phs_cblock;
+ char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelHeapScanDescData;
typedef struct HeapScanDescData
{
@@ -49,6 +58,7 @@ typedef struct HeapScanDescData
BlockNumber rs_cblock; /* current block # in scan, if any */
Buffer rs_cbuf; /* current buffer in scan, if any */
/* NB: if rs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
+ ParallelHeapScanDesc rs_parallel; /* parallel scan information */
/* these fields only used in page-at-a-time mode and for bitmap scans */
int rs_cindex; /* current tuple's index in vistuples */
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
new file mode 100644
index 0000000..73006a8
--- /dev/null
+++ b/src/include/executor/execParallel.h
@@ -0,0 +1,65 @@
+/*--------------------------------------------------------------------
+ * execParallel.h
+ * POSTGRES backend workers interface
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/executor/execParallel.h
+ *--------------------------------------------------------------------
+ */
+#ifndef EXECPARALLEL_H
+#define EXECPARALLEL_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+#include "nodes/execnodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_PARAMS_EXEC 2
+#define PARALLEL_KEY_BUFF_USAGE 3
+#define PARALLEL_KEY_INST_OPTIONS 4
+#define PARALLEL_KEY_INST_INFO 5
+#define PARALLEL_KEY_TUPLE_QUEUE 6
+#define PARALLEL_KEY_SCAN 7
+
+extern int parallel_seqscan_degree;
+
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
+
+extern void InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+extern shm_toc *GetParallelShmToc(void);
+extern bool ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelBufferUsageAccum(Node *node);
+extern void ExecAssociateBufferStatsToDSM(BufferUsage *buf_usage,
+ ParallelStmt *parallel_stmt);
+#endif /* EXECPARALLEL_H */
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 193a654..963e656 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -273,6 +273,8 @@ extern TupleDesc ExecCleanTypeFromTL(List *targetList, bool hasoid);
extern TupleDesc ExecTypeFromExprList(List *exprList);
extern void ExecTypeSetColNames(TupleDesc typeInfo, List *namesList);
extern void UpdateChangedParamSet(PlanState *node, Bitmapset *newchg);
+extern void PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals);
typedef struct TupOutputState
{
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index c9a2129..0c7847d 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..27d0b3d
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void FinishParallelSetupAndAccumStats(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..47b8f73
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ *
+ *
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node,
+ EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/executor/nodeSubplan.h b/src/include/executor/nodeSubplan.h
index 3732ad4..21c745e 100644
--- a/src/include/executor/nodeSubplan.h
+++ b/src/include/executor/nodeSubplan.h
@@ -24,4 +24,7 @@ extern void ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent);
extern void ExecSetParamPlan(SubPlanState *node, ExprContext *econtext);
+extern List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params);
+
#endif /* NODESUBPLAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..d2ddb6e
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,35 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void TupleQueueFunnelShutdown(TupleQueueFunnel *funnel);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 541ee18..cc1174e 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
@@ -401,6 +403,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1050,6 +1064,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1267,6 +1286,45 @@ typedef struct SampleScanState
} SampleScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodeFuncs.h b/src/include/nodes/nodeFuncs.h
index 7b1b1d6..df00d3d 100644
--- a/src/include/nodes/nodeFuncs.h
+++ b/src/include/nodes/nodeFuncs.h
@@ -13,6 +13,7 @@
#ifndef NODEFUNCS_H
#define NODEFUNCS_H
+#include "access/parallel.h"
#include "nodes/parsenodes.h"
@@ -63,4 +64,7 @@ extern Node *query_or_expression_tree_mutator(Node *node, Node *(*mutator) (),
extern bool raw_expression_tree_walker(Node *node, bool (*walker) (),
void *context);
+extern bool planstate_tree_walker(Node *node, ParallelContext *pcxt,
+ bool (*walker) (), void *context);
+
#endif /* NODEFUNCS_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 290cdb3..322b5e8 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -51,6 +51,8 @@ typedef enum NodeTag
T_BitmapOr,
T_Scan,
T_SeqScan,
+ T_PartialSeqScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -99,6 +101,8 @@ typedef enum NodeTag
T_ScanState,
T_SeqScanState,
T_SampleScanState,
+ T_PartialSeqScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -223,6 +227,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..21c6f7a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -14,6 +14,8 @@
#ifndef PARAMS_H
#define PARAMS_H
+#include "nodes/pg_list.h"
+
/* To avoid including a pile of parser headers, reference ParseState thus: */
struct ParseState;
@@ -96,11 +98,47 @@ typedef struct ParamExecData
{
void *execPlan; /* should be "SubPlanState *" */
Datum value;
+ /*
+ * parameter's datatype, or 0. This is required so that
+ * datum value can be read and used for other purposes like
+ * passing it to worker backend via shared memory. This is
+ * required only for evaluation of initPlan's, however for
+ * consistency we set this for Subplan as well. We left it
+ * for other cases like CTE or RecursiveUnion cases where this
+ * structure is not used for evaluation of subplans.
+ */
+ Oid ptype;
bool isnull;
} ParamExecData;
+/*
+ * This structure is used to pass PARAM_EXEC parameters to backend
+ * workers. For each PARAM_EXEC parameter, pass this structure
+ * followed by value except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExecData
+{
+ int paramid; /* parameter id of this param */
+ Size length; /* length of parameter value */
+ Oid ptype; /* parameter's datatype, or 0 */
+ Datum value;
+ bool isnull;
+} SerializedParamExecData;
+
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
+extern Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals);
+extern void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address);
+List *
+RestoreExecParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 761bdf4..5d705c1 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -292,6 +292,22 @@ typedef Scan SeqScan;
typedef Scan SampleScan;
/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
+/* ----------------
+ * parallel sequential scan node
+ * ----------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
+/* ----------------
* index scan node
*
* indexqualorig is an implicitly-ANDed list of index qual expressions, each
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 53a8820..62b498b 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -754,6 +754,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 24003ae..a1c9f59 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,13 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +55,11 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -69,6 +79,11 @@ extern double index_pages_fetched(double tuples_fetched, BlockNumber pages,
extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel);
+extern void cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 161644c..6047fec 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -34,6 +34,10 @@ extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
extern Path *create_samplescan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, int nworkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 3e2378a..bd8eb67 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -55,6 +55,13 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel);
+
+/*
* indxpath.c
* routines to generate index paths
*/
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 52b077a..67a8582 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -133,6 +133,7 @@ extern bool query_is_distinct_for(Query *query, List *colnos, List *opids);
*/
extern Plan *set_plan_references(PlannerInfo *root, Plan *plan);
extern void fix_opfuncids(Node *node);
+extern void fix_node_funcids(Plan *node);
extern void set_opfuncid(OpExpr *opexpr);
extern void set_sa_opfuncid(ScalarArrayOpExpr *opexpr);
extern void record_plan_function_dependency(PlannerInfo *root, Oid funcid);
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..8c7ce75 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(PartialSeqScan *partialscan,
+ List *rangetable, int num_exec_params);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/storage/shm_mq.h b/src/include/storage/shm_mq.h
index 1a2ba04..7621a35 100644
--- a/src/include/storage/shm_mq.h
+++ b/src/include/storage/shm_mq.h
@@ -65,6 +65,9 @@ extern void shm_mq_set_handle(shm_mq_handle *, BackgroundWorkerHandle *);
/* Break connection. */
extern void shm_mq_detach(shm_mq *);
+/* Get the shm_mq from handle. */
+extern shm_mq *shm_mq_get_queue(shm_mq_handle *mqh);
+
/* Send or receive messages. */
extern shm_mq_result shm_mq_send(shm_mq_handle *mqh,
Size nbytes, const void *data, bool nowait);
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..91acd60 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
@@ -103,7 +104,9 @@ typedef enum
* pointers that the executor must call.
*
* Note: the receiveSlot routine must be passed a slot containing a TupleDesc
- * identical to the one given to the rStartup routine.
+ * identical to the one given to the rStartup routine. It returns bool where
+ * a "true" value means "continue processing" and a "false" value means
+ * "stop early, just as if we'd reached the end of the scan".
* ----------------
*/
typedef struct _DestReceiver DestReceiver;
@@ -111,7 +114,7 @@ typedef struct _DestReceiver DestReceiver;
struct _DestReceiver
{
/* Called for each tuple to be output: */
- void (*receiveSlot) (TupleTableSlot *slot,
+ bool (*receiveSlot) (TupleTableSlot *slot,
DestReceiver *self);
/* Per-executor-run initialization and shutdown: */
void (*rStartup) (DestReceiver *self,
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 96c5b8b..6f319c1 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -19,6 +19,7 @@
#ifndef TCOPPROT_H
#define TCOPPROT_H
+#include "executor/execParallel.h"
#include "nodes/params.h"
#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
@@ -84,5 +85,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index 7a58ddb..3505d31 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Jul 6, 2015 at 3:26 AM, Jeff Davis <pgsql@j-davis.com> wrote:
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?
The main reason of changing heapam.c is that we want to scan blocks
parallely by multiple workers and heapam.c seems to be the best
place to make such a change. As of now, the changes are mainly
required to identify the next block to scan by each worker. So
we can focus on that aspect and see if anything is missing.
* 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?
Yes, there is a correctness issue. All the parallel workers should see
the same scan information during scan as is seen by master backend.
master backend fills this structure and then that is used by all workers
to avoid any problem.
Is initscan expecting that heap_parallelscan_initialize is always called
first (if parallel)? Please add a comment explaining above.
okay.
* What's the difference between scan->rs_nblocks and
scan->rs_parallel->phs_nblocks?
scan->rs_parallel->phs_nblocks is once initialized in master
backend and then propagated to all other worker backends and
then worker backends use that value to initialize scan->rs_nblocks
(and if master backend itself is involved in scan, then it also
uses it in same way)
Same for rs_rd->rd_id and phs_relid.
This is also similar to phs_nblocks. The basic idea is that parallel
heap scan descriptor is formed in master backend containing all the
necessary members that are required for performing the scan in master
as well as worker backends. Once we initialize the parallel heap scan
descriptor, it is passed to all the worker backends and used by them
to scan the heap.
* 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.
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.
* 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.
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.
If we don't want to mix
syncscan and partial scan, that's fine, but it should be more explicit.
makes sense to me, I think in initscan, we should mark syncscan
as false for parallel scan case.
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.
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.
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.
It has been designed to be generic way of communication for tuples,
but let me know if you have any specific suggestions.
I still have quite a bit to look at, but this is a start.
Thanks for the review.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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:
/messages/by-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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, Jul 6, 2015 at 10:54 PM, Jeff Davis <pgsql@j-davis.com> wrote:
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.
In patch rs_nblocks is used in paratial scan's as well, only the
way to initialize is changed.
Or, you could try wrapping the parts of heapam.c that are affected by
parallelism into new static functions.
Sounds sensible to me, but I would like to hear from Robert before
making this change, if he has any different opinions about this point, as
he has originally written this part of the patch.
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.
I am not sure how many blocks difference could be considered okay for
deviation?
In theory, making parallel scan perform sync scan could lead to difference
of multiple blocks, consider the case where there are 32 or more workers
participating in scan and each got one block to scan, it is possible that
first worker performs scan of 1st block after 32nd worker performs the
scan of 32nd block (it could lead to even bigger differences).
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).
Okay, this makes sense to me, I think it is better to move Funnel
operator specific parts out of tqueue.c unless Robert or anybody else
feels otherwise.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Jul 7, 2015 at 6:19 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
wrote:
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:
/messages/by-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.
For reasonable default values for these parameters, still more testing
is required. I think instead of 0, tests with 0.001 or 0.0025 for default
of cpu_tuple_comm_cost and 100 or 1000 for default of parallel_setup_cost
would have been more interesting.
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.
These are quite similar to what I have seen in my initial tests, now I
think if you add some complex condition in the filter, you will see gains
for even 25% or more selectivity (I have added factorial 10 calculation in
filter to mimic the complex filter condition).
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Wed, Jul 15, 2015 at 2:14 PM, Antonin Houska <ah@cybertec.at> wrote:
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.
Right, I think this is left over from initial version where parallel seq
scan
was supported just for single table scan. It should probably do similar to
create_seqscan_path() and then pass the same down to
create_partialseqscan_path() and get_baserel_parampathinfo().
Thanks, I will fix this in next version of patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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 and
ignore 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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, 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.
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
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.
Thanks for looking into patch!
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, 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.27The 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
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 ScanOn 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/>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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>
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
Import Notes
Resolved by subject fallback
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 ScanOn 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sun, Aug 2, 2015 at 8:06 AM, Kouhei Kaigai <kaigai@ak.jp.nec.com> wrote:
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)?
The idea is that if in some case the qualification can't be
pushed down (consider the case where qualification contains
parallel restricted functions (functions that can only be
executed in master backend)) and needs to be only executed
in master backend, then we need it in Funnel node, so that it
can be executed for tuples passed by worker backends. It is
currently not used, but I think we should retain it as it is
because it can be used in some cases either as part of this
patch itself or in future. As of now, it is used in other
places in patch (like during Explain) as well, although we
might want to optimize the same, but overall I think it is
required.
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.
It needs it's own target-list due to reason mentioned above
for qual and yet another reason is that the same is required
for FunnelState which inturn is required ScanSlot used to
retrieve tuples from workers. Also it is not excatly same
as partialseqscan, because for the case when the partialseqscan
node is executed by worker, we modify the targetlist as well,
refer create_parallel_worker_plannedstmt().
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[].
Isn't EvalPlanQual() called for modifytable node and which
won't be allowed in parallel mode, so I think EvalPlanQual()
is not required for ExecFunnel path.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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.
Note - To test the patch, you need to first apply the assess-parallel-safety
patch [1]/messages/by-id/CAA4eK1Kd2SunKX=e5sSFSrFfc++_uHnt5_HyKd+XykFjDWZseQ@mail.gmail.com and then apply parallel_seqscan_funnel_v17.patch attached with
this mail and then apply parallel_seqscan_partialseqscan_v17.patch attached
with this mail.
[1]: /messages/by-id/CAA4eK1Kd2SunKX=e5sSFSrFfc++_uHnt5_HyKd+XykFjDWZseQ@mail.gmail.com
/messages/by-id/CAA4eK1Kd2SunKX=e5sSFSrFfc++_uHnt5_HyKd+XykFjDWZseQ@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
Attachments:
parallel_seqscan_funnel_v17.patchapplication/octet-stream; name=parallel_seqscan_funnel_v17.patchDownload
diff --git a/src/backend/access/common/printtup.c b/src/backend/access/common/printtup.c
index baed981..639451a 100644
--- a/src/backend/access/common/printtup.c
+++ b/src/backend/access/common/printtup.c
@@ -26,9 +26,9 @@
static void printtup_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-static void printtup(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_20(TupleTableSlot *slot, DestReceiver *self);
-static void printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_20(TupleTableSlot *slot, DestReceiver *self);
+static bool printtup_internal_20(TupleTableSlot *slot, DestReceiver *self);
static void printtup_shutdown(DestReceiver *self);
static void printtup_destroy(DestReceiver *self);
@@ -299,7 +299,7 @@ printtup_prepare_info(DR_printtup *myState, TupleDesc typeinfo, int numAttrs)
* printtup --- print a tuple in protocol 3.0
* ----------------
*/
-static void
+static bool
printtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -376,13 +376,15 @@ printtup(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
* printtup_20 --- print a tuple in protocol 2.0
* ----------------
*/
-static void
+static bool
printtup_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -452,6 +454,8 @@ printtup_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
/* ----------------
@@ -528,7 +532,7 @@ debugStartup(DestReceiver *self, int operation, TupleDesc typeinfo)
* debugtup - print one tuple for an interactive backend
* ----------------
*/
-void
+bool
debugtup(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -553,6 +557,8 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
printatt((unsigned) i + 1, typeinfo->attrs[i], value);
}
printf("\t----\n");
+
+ return true;
}
/* ----------------
@@ -564,7 +570,7 @@ debugtup(TupleTableSlot *slot, DestReceiver *self)
* This is largely same as printtup_20, except we use binary formatting.
* ----------------
*/
-static void
+static bool
printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
{
TupleDesc typeinfo = slot->tts_tupleDescriptor;
@@ -636,4 +642,6 @@ printtup_internal_20(TupleTableSlot *slot, DestReceiver *self)
/* Return to caller's context, and flush row's temporary memory */
MemoryContextSwitchTo(oldcontext);
MemoryContextReset(myState->tmpcontext);
+
+ return true;
}
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8db1b35..b55c4dc 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -4414,7 +4414,7 @@ copy_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* copy_dest_receive --- receive one tuple
*/
-static void
+static bool
copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_copy *myState = (DR_copy *) self;
@@ -4426,6 +4426,8 @@ copy_dest_receive(TupleTableSlot *slot, DestReceiver *self)
/* And send the data */
CopyOneRowTo(cstate, InvalidOid, slot->tts_values, slot->tts_isnull);
myState->processed++;
+
+ return true;
}
/*
diff --git a/src/backend/commands/createas.c b/src/backend/commands/createas.c
index 41183f6..418b0f6 100644
--- a/src/backend/commands/createas.c
+++ b/src/backend/commands/createas.c
@@ -62,7 +62,7 @@ typedef struct
static ObjectAddress CreateAsReladdr = {InvalidOid, InvalidOid, 0};
static void intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void intorel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool intorel_receive(TupleTableSlot *slot, DestReceiver *self);
static void intorel_shutdown(DestReceiver *self);
static void intorel_destroy(DestReceiver *self);
@@ -482,7 +482,7 @@ intorel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* intorel_receive --- receive one tuple
*/
-static void
+static bool
intorel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_intorel *myState = (DR_intorel *) self;
@@ -507,6 +507,8 @@ intorel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 4f32400..69d3b34 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -20,6 +20,7 @@
#include "commands/defrem.h"
#include "commands/prepare.h"
#include "executor/hashjoin.h"
+#include "executor/nodeFunnel.h"
#include "foreign/fdwapi.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
@@ -733,6 +734,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
{
case T_SeqScan:
case T_SampleScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -940,6 +942,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SampleScan:
pname = sname = "Sample Scan";
break;
+ case T_Funnel:
+ pname = sname = "Funnel";
+ break;
case T_IndexScan:
pname = sname = "Index Scan";
break;
@@ -1090,6 +1095,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
{
case T_SeqScan:
case T_SampleScan:
+ case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
case T_SubqueryScan:
@@ -1234,6 +1240,16 @@ ExplainNode(PlanState *planstate, List *ancestors,
}
/*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. Though we already accumulate this
+ * information when last tuple is fetched from Funnel node, this
+ * is to cover cases when we don't fetch all tuples from a node
+ * such as for Limit node.
+ */
+ if (es->analyze && nodeTag(plan) == T_Funnel)
+ FinishParallelSetupAndAccumStats((FunnelState *)planstate);
+
+ /*
* We have to forcibly clean up the instrumentation state because we
* haven't done ExecutorEnd yet. This is pretty grotty ...
*
@@ -1363,6 +1379,14 @@ ExplainNode(PlanState *planstate, List *ancestors,
show_instrumentation_count("Rows Removed by Filter", 1,
planstate, es);
break;
+ case T_Funnel:
+ 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",
+ ((Funnel *) plan)->num_workers, es);
+ break;
case T_FunctionScan:
if (es->verbose)
{
@@ -2422,6 +2446,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
{
case T_SeqScan:
case T_SampleScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
diff --git a/src/backend/commands/matview.c b/src/backend/commands/matview.c
index 5492e59..750a59c 100644
--- a/src/backend/commands/matview.c
+++ b/src/backend/commands/matview.c
@@ -56,7 +56,7 @@ typedef struct
static int matview_maintenance_depth = 0;
static void transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool transientrel_receive(TupleTableSlot *slot, DestReceiver *self);
static void transientrel_shutdown(DestReceiver *self);
static void transientrel_destroy(DestReceiver *self);
static void refresh_matview_datafill(DestReceiver *dest, Query *query,
@@ -422,7 +422,7 @@ transientrel_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* transientrel_receive --- receive one tuple
*/
-static void
+static bool
transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_transientrel *myState = (DR_transientrel *) self;
@@ -441,6 +441,8 @@ transientrel_receive(TupleTableSlot *slot, DestReceiver *self)
myState->bistate);
/* We know this is a newly created relation, so there are no indexes */
+
+ return true;
}
/*
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index 08cba6f..8037417 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -13,17 +13,17 @@ top_builddir = ../../..
include $(top_builddir)/src/Makefile.global
OBJS = execAmi.o execCurrent.o execGrouping.o execIndexing.o execJunk.o \
- execMain.o execProcnode.o execQual.o execScan.o execTuples.o \
+ execMain.o execParallel.o execProcnode.o execQual.o execScan.o execTuples.o \
execUtils.o functions.o instrument.o nodeAppend.o nodeAgg.o \
nodeBitmapAnd.o nodeBitmapOr.o \
- nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeHash.o \
- nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
+ nodeBitmapHeapscan.o nodeBitmapIndexscan.o nodeCustom.o nodeFunnel.o \
+ nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
- nodeForeignscan.o nodeWindowAgg.o tstoreReceiver.o spi.o
+ nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o spi.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 93e1e9a..4915151 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -24,6 +24,7 @@
#include "executor/nodeCustom.h"
#include "executor/nodeForeignscan.h"
#include "executor/nodeFunctionscan.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeGroup.h"
#include "executor/nodeGroup.h"
#include "executor/nodeHash.h"
@@ -160,6 +161,10 @@ ExecReScan(PlanState *node)
ExecReScanSampleScan((SampleScanState *) node);
break;
+ case T_FunnelState:
+ ExecReScanFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecReScanIndexScan((IndexScanState *) node);
break;
@@ -467,6 +472,9 @@ ExecSupportsBackwardScan(Plan *node)
/* Simplify life for tablesample methods by disallowing this */
return false;
+ case T_Funnel:
+ return false;
+
case T_IndexScan:
return IndexSupportsBackwardScan(((IndexScan *) node)->indexid) &&
TargetListSupportsBackwardScan(node->targetlist);
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index bcd287f..650fcc5 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -262,6 +262,7 @@ search_plan_tree(PlanState *node, Oid table_oid)
*/
case T_SeqScanState:
case T_SampleScanState:
+ case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
case T_BitmapHeapScanState:
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index b62e88b..e35cbbe 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -45,9 +45,11 @@
#include "commands/matview.h"
#include "commands/trigger.h"
#include "executor/execdebug.h"
+#include "executor/execParallel.h"
#include "foreign/fdwapi.h"
#include "mb/pg_wchar.h"
#include "miscadmin.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "parser/parsetree.h"
#include "storage/bufmgr.h"
@@ -323,6 +325,9 @@ standard_ExecutorRun(QueryDesc *queryDesc,
operation = queryDesc->operation;
dest = queryDesc->dest;
+ /* inform executor to collect buffer usage stats from parallel workers. */
+ estate->total_time = queryDesc->totaltime ? 1 : 0;
+
/*
* startup tuple receiver, if we will be emitting tuples
*/
@@ -354,7 +359,15 @@ standard_ExecutorRun(QueryDesc *queryDesc,
(*dest->rShutdown) (dest);
if (queryDesc->totaltime)
+ {
+ /*
+ * Accumulate the stats by parallel workers before stopping the
+ * node.
+ */
+ (void) planstate_tree_walker((Node*) queryDesc->planstate,
+ NULL, ExecParallelBufferUsageAccum, 0);
InstrStopNode(queryDesc->totaltime, estate->es_processed);
+ }
MemoryContextSwitchTo(oldcontext);
}
@@ -1581,7 +1594,15 @@ ExecutePlan(EState *estate,
* practice, this is probably always the case at this point.)
*/
if (sendTuples)
- (*dest->receiveSlot) (slot, dest);
+ {
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
+ }
/*
* Count tuples processed, if this is a SELECT. (For other operation
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
new file mode 100644
index 0000000..806f060
--- /dev/null
+++ b/src/backend/executor/execParallel.c
@@ -0,0 +1,559 @@
+/*-------------------------------------------------------------------------
+ *
+ * execParallel.c
+ * Support routines for setting up backend workers for parallel execution.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/execParallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "nodes/nodeFuncs.h"
+#include "optimizer/planmain.h"
+#include "optimizer/planner.h"
+#include "tcop/tcopprot.h"
+
+
+#define PARALLEL_TUPLE_QUEUE_SIZE 65536
+
+static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+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);
+static void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size);
+static void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size);
+static void EstimateResponseQueueSpace(ParallelContext *pcxt);
+static void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp);
+static void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt);
+static void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage);
+static void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq);
+
+
+/*
+ * This is required for parallel plan execution to fetch the information
+ * from dsm.
+ */
+static shm_toc *parallel_shm_toc = NULL;
+
+/*
+ * EstimateParallelSupportInfoSpace
+ *
+ * Estimate the amount of space required to record information of bind
+ * parameters, PARAM_EXEC parameters and instrumentation information that
+ * need to be retrieved from parallel workers.
+ */
+void
+EstimateParallelSupportInfoSpace(ParallelContext *pcxt, ParamListInfo params,
+ List *serialized_param_exec_vals,
+ int instOptions, Size *params_size,
+ Size *params_exec_size)
+{
+ *params_size = EstimateBoundParametersSpace(params);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_size);
+
+ *params_exec_size = EstimateExecParametersSpace(serialized_param_exec_vals);
+ shm_toc_estimate_chunk(&pcxt->estimator, *params_exec_size);
+
+ /*
+ * 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.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(BufferUsage) * pcxt->nworkers);
+
+ /* account for instrumentation options. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(int));
+
+ /*
+ * 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.
+ */
+ if (instOptions)
+ {
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ sizeof(Instrumentation) * pcxt->nworkers);
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* keys for parallel support information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 4);
+}
+
+/*
+ * StoreParallelSupportInfo
+ *
+ * Sets up the bind parameters, PARAM_EXEC parameters and instrumentation
+ * information required for parallel execution.
+ */
+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)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ int *inst_options;
+
+ /*
+ * Store bind parameter's list in dynamic shared memory. This is
+ * used for parameters in prepared query.
+ */
+ paramsdata = shm_toc_allocate(pcxt->toc, params_size);
+ SerializeBoundParams(params, params_size, paramsdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS, paramsdata);
+
+ /*
+ * Store PARAM_EXEC parameters list in dynamic shared memory. This is
+ * used for evaluation plan->initPlan params.
+ */
+ paramsexecdata = shm_toc_allocate(pcxt->toc, params_exec_size);
+ SerializeExecParams(serialized_param_exec_vals, params_exec_size, paramsexecdata);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PARAMS_EXEC, paramsexecdata);
+
+ /*
+ * Allocate space for BufferUsage information to be filled by
+ * each worker.
+ */
+ *buffer_usage_space =
+ shm_toc_allocate(pcxt->toc, sizeof(BufferUsage) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_BUFF_USAGE, *buffer_usage_space);
+
+ /* Store instrument options in dynamic shared memory. */
+ inst_options = shm_toc_allocate(pcxt->toc, sizeof(int));
+ *inst_options = instOptions;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_OPTIONS, inst_options);
+
+ /*
+ * Allocate space for instrumentation information to be filled by
+ * each worker.
+ */
+ if (instOptions)
+ {
+ *inst_options_space =
+ shm_toc_allocate(pcxt->toc, sizeof(Instrumentation) * pcxt->nworkers);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_INST_INFO, *inst_options_space);
+ }
+}
+
+/*
+ * EstimatePlannedStmtSpace
+ *
+ * Estimate the amount of space required to record information of planned
+ * statement and parallel node specific information that need to be copied
+ * to parallel workers.
+ */
+void
+EstimatePlannedStmtSpace(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size *plannedstmt_len,
+ Size *pscan_size)
+{
+ /* Estimate space for planned statement. */
+ *plannedstmt_len = strlen(plannedstmt_str) + 1;
+ shm_toc_estimate_chunk(&pcxt->estimator, *plannedstmt_len);
+
+ /* keys for planned statement information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelEstimate,
+ pscan_size);
+}
+
+/*
+ * StorePlannedStmt
+ *
+ * Sets up the planned statement and node specific information.
+ */
+void
+StorePlannedStmt(ParallelContext *pcxt, PlanState* planstate,
+ char *plannedstmt_str, Size plannedstmt_size,
+ Size pscan_size)
+{
+ char *plannedstmtdata;
+
+ /* Store planned statement in dynamic shared memory. */
+ plannedstmtdata = shm_toc_allocate(pcxt->toc, plannedstmt_size);
+ memcpy(plannedstmtdata, plannedstmt_str, plannedstmt_size);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_PLANNEDSTMT, plannedstmtdata);
+
+ (void) planstate_tree_walker((Node*)planstate, pcxt, ExecParallelInitializeDSM,
+ &pscan_size);
+}
+
+/*
+ * EstimateResponseQueueSpace
+ *
+ * Estimate the amount of space required to record information of tuple
+ * queues that need to be established between parallel workers and master
+ * backend.
+ */
+void
+EstimateResponseQueueSpace(ParallelContext *pcxt)
+{
+ /* Estimate space for tuple queues. */
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+
+ /* keys for response queue. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+}
+
+/*
+ * StoreResponseQueue
+ *
+ * It sets up the response queues for backend workers to return tuples
+ * to the main backend and start the workers.
+ */
+void
+StoreResponseQueue(ParallelContext *pcxt,
+ shm_mq_handle ***responseqp)
+{
+ shm_mq *mq;
+ char *tuple_queue_space;
+ int i;
+
+ /* Allocate memory for shared memory queue handles. */
+ *responseqp = (shm_mq_handle**) palloc(pcxt->nworkers * sizeof(shm_mq_handle*));
+
+ /*
+ * Establish one message queue per worker in dynamic shared memory.
+ * These queues should be used to transmit tuple data.
+ */
+ tuple_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_TUPLE_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ mq = shm_mq_create(tuple_queue_space + i * PARALLEL_TUPLE_QUEUE_SIZE,
+ (Size) PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_receiver(mq, MyProc);
+
+ /*
+ * Attach the queue before launching a worker, so that we'll automatically
+ * detach the queue if we error out. Otherwise, the worker might sit
+ * there trying to write the queue long after we've gone away.
+ */
+ (*responseqp)[i] = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TUPLE_QUEUE, tuple_queue_space);
+}
+
+/*
+ * ExecParallelEstimate
+ *
+ * Estimate the amount of space required to record information of
+ * parallel node that need to be copied to parallel workers.
+ */
+bool
+ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ if (node == NULL)
+ return false;
+
+ /*
+ * As of now, we support few nodes that can be passed to parallel
+ * workers, so handle only those nodes.
+ */
+ switch (nodeTag(node))
+ {
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * ExecParallelInitializeDSM
+ *
+ * Store the information of parallel node in dsm.
+ */
+bool
+ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size)
+{
+ if (node == NULL)
+ return false;
+
+ /*
+ * As of now, we support few nodes that can be passed to parallel
+ * workers, so handle only those nodes.
+ */
+ switch (nodeTag(node))
+ {
+ default:
+ break;
+ }
+
+ return false;
+}
+
+/*
+ * InitializeParallelWorkers
+ *
+ * Sets up the required infrastructure for backend workers to perform
+ * execution and return results to the main backend.
+ */
+void
+InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers)
+{
+ Size params_size, params_exec_size, pscan_size, plannedstmt_size;
+ char *plannedstmt_str;
+ PlannedStmt *plannedstmt;
+ ParallelContext *pcxt;
+
+ pcxt = CreateParallelContext(ParallelQueryMain, nWorkers);
+
+ plannedstmt = create_parallel_worker_plannedstmt(planstate->plan,
+ estate->es_range_table,
+ estate->es_plannedstmt->nParamExec);
+ plannedstmt_str = nodeToString(plannedstmt);
+
+ EstimatePlannedStmtSpace(pcxt, planstate, plannedstmt_str,
+ &plannedstmt_size, &pscan_size);
+ EstimateParallelSupportInfoSpace(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument, ¶ms_size,
+ ¶ms_exec_size);
+ EstimateResponseQueueSpace(pcxt);
+
+ InitializeParallelDSM(pcxt);
+
+ StorePlannedStmt(pcxt, planstate, plannedstmt_str,
+ plannedstmt_size, pscan_size);
+ StoreParallelSupportInfo(pcxt, estate->es_param_list_info,
+ serialized_param_exec_vals,
+ estate->es_instrument,
+ params_size,
+ params_exec_size,
+ inst_options_space,
+ buffer_usage_space);
+ StoreResponseQueue(pcxt, responseqp);
+
+ /* Return results to caller. */
+ *pcxtp = pcxt;
+}
+
+/*
+ * GetParallelSupportInfo
+ *
+ * Look up based on keys in dynamic shared memory segment and get the
+ * bind parameters, PARAM_EXEC parameters and instrumentation information
+ * required to perform parallel operation.
+ */
+void
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
+ List **serialized_param_exec_vals,
+ int *inst_options, char **instrument,
+ char **buffer_usage)
+{
+ char *paramsdata;
+ char *paramsexecdata;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ int *instoptions;
+
+ if (params)
+ {
+ paramsdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS);
+ *params = RestoreBoundParams(paramsdata);
+ }
+
+ if (serialized_param_exec_vals)
+ {
+ paramsexecdata = shm_toc_lookup(toc, PARALLEL_KEY_PARAMS_EXEC);
+ *serialized_param_exec_vals = RestoreExecParams(paramsexecdata);
+ }
+
+ if (inst_options)
+ {
+ instoptions = shm_toc_lookup(toc, PARALLEL_KEY_INST_OPTIONS);
+ *inst_options = *instoptions;
+ if (inst_options)
+ {
+ inst_options_space = shm_toc_lookup(toc, PARALLEL_KEY_INST_INFO);
+ *instrument = (inst_options_space +
+ ParallelWorkerNumber * sizeof(Instrumentation));
+ }
+ }
+
+ if (buffer_usage)
+ {
+ buffer_usage_space = shm_toc_lookup(toc, PARALLEL_KEY_BUFF_USAGE);
+ *buffer_usage = (buffer_usage_space +
+ ParallelWorkerNumber * sizeof(BufferUsage));
+ }
+}
+
+/*
+ * ExecParallelGetPlannedStmt
+ *
+ * Look up based on keys in dynamic shared memory segment and get the
+ * planned statement required to perform parallel operation.
+ */
+void
+ExecParallelGetPlannedStmt(shm_toc *toc, PlannedStmt **plannedstmt)
+{
+ char *plannedstmtdata;
+
+ plannedstmtdata = shm_toc_lookup(toc, PARALLEL_KEY_PLANNEDSTMT);
+
+ *plannedstmt = (PlannedStmt *) stringToNode(plannedstmtdata);
+
+ /* Fill in opfuncid values if missing */
+ fix_node_funcids((*plannedstmt)->planTree);
+}
+
+/*
+ * SetupResponseQueue
+ *
+ * Look up based on keys in dynamic shared memory segment and get the
+ * tuple queue information for a particular worker, attach to the queue
+ * and redirect all futher responses from worker backend via that queue.
+ */
+void
+SetupResponseQueue(dsm_segment *seg, shm_toc *toc, shm_mq **mq,
+ shm_mq_handle **responseq)
+{
+ char *tuple_queue_space;
+
+ tuple_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_TUPLE_QUEUE);
+ *mq = (shm_mq *) (tuple_queue_space +
+ ParallelWorkerNumber * PARALLEL_TUPLE_QUEUE_SIZE);
+
+ shm_mq_set_sender(*mq, MyProc);
+ *responseq = shm_mq_attach(*mq, seg, NULL);
+}
+
+/*
+ * GetParallelShmToc
+ */
+shm_toc *
+GetParallelShmToc(void)
+{
+ return parallel_shm_toc;
+}
+
+/*
+ * ParallelQueryMain
+ *
+ * Execute the operation to return the tuples or other information to
+ * parallelism driving node.
+ */
+void
+ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
+{
+ shm_mq *mq;
+ shm_mq_handle *responseq;
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ int inst_options;
+ char *instrument = NULL;
+ char *buffer_usage = NULL;
+ ParallelStmt *parallelstmt;
+
+ SetupResponseQueue(seg, toc, &mq, &responseq);
+
+ ExecParallelGetPlannedStmt(toc, &plannedstmt);
+ GetParallelSupportInfo(toc, ¶ms, &serialized_param_exec_vals,
+ &inst_options, &instrument, &buffer_usage);
+
+ parallelstmt = palloc(sizeof(ParallelStmt));
+
+ parallelstmt->plannedstmt = plannedstmt;
+ parallelstmt->params = params;
+ parallelstmt->serialized_param_exec_vals = serialized_param_exec_vals;
+ parallelstmt->inst_options = inst_options;
+ parallelstmt->instrument = instrument;
+ parallelstmt->buffer_usage = buffer_usage;
+ parallelstmt->responseq = responseq;
+
+ parallel_shm_toc = toc;
+
+ /* Execute the worker command. */
+ exec_parallel_stmt(parallelstmt);
+
+ /*
+ * Once we are done with sending tuples, detach from shared memory
+ * message queue used to send tuples.
+ */
+ shm_mq_detach(mq);
+}
+
+/*
+ * ExecParallelBufferUsageAccum
+ *
+ * Recursively accumulate the stats for all the funnel nodes in a plan
+ * state tree.
+ */
+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;
+}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index 03c2feb..c181bf2 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -196,6 +197,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_Funnel:
+ result = (PlanState *) ExecInitFunnel((Funnel *) node,
+ estate, eflags);
+ break;
+
case T_IndexScan:
result = (PlanState *) ExecInitIndexScan((IndexScan *) node,
estate, eflags);
@@ -416,6 +422,10 @@ ExecProcNode(PlanState *node)
result = ExecSampleScan((SampleScanState *) node);
break;
+ case T_FunnelState:
+ result = ExecFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
result = ExecIndexScan((IndexScanState *) node);
break;
@@ -658,6 +668,10 @@ ExecEndNode(PlanState *node)
ExecEndSampleScan((SampleScanState *) node);
break;
+ case T_FunnelState:
+ ExecEndFunnel((FunnelState *) node);
+ break;
+
case T_IndexScanState:
ExecEndIndexScan((IndexScanState *) node);
break;
diff --git a/src/backend/executor/execTuples.c b/src/backend/executor/execTuples.c
index a05d8b1..d5619bd 100644
--- a/src/backend/executor/execTuples.c
+++ b/src/backend/executor/execTuples.c
@@ -1313,7 +1313,7 @@ do_tup_output(TupOutputState *tstate, Datum *values, bool *isnull)
ExecStoreVirtualTuple(slot);
/* send the tuple to the receiver */
- (*tstate->dest->receiveSlot) (slot, tstate->dest);
+ (void) (*tstate->dest->receiveSlot) (slot, tstate->dest);
/* clean up */
ExecClearTuple(slot);
diff --git a/src/backend/executor/execUtils.c b/src/backend/executor/execUtils.c
index 3c611b9..a0d6441 100644
--- a/src/backend/executor/execUtils.c
+++ b/src/backend/executor/execUtils.c
@@ -976,3 +976,27 @@ ShutdownExprContext(ExprContext *econtext, bool isCommit)
MemoryContextSwitchTo(oldcontext);
}
+
+/*
+ * Populate the values of PARAM_EXEC parameters.
+ *
+ * This is used by worker backends to fill in the values of PARAM_EXEC
+ * parameters after fetching the same from dynamic shared memory.
+ * This needs to be called before ExecutorRun.
+ */
+void
+PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals)
+{
+ ListCell *lparam;
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].value =
+ param_val->value;
+ queryDesc->estate->es_param_exec_vals[param_val->paramid].isnull =
+ param_val->isnull;
+ }
+}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 812a610..863bd64 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -167,7 +167,7 @@ static Datum postquel_get_single_result(TupleTableSlot *slot,
static void sql_exec_error_callback(void *arg);
static void ShutdownSQLFunction(Datum arg);
static void sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo);
-static void sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
+static bool sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self);
static void sqlfunction_shutdown(DestReceiver *self);
static void sqlfunction_destroy(DestReceiver *self);
@@ -1903,7 +1903,7 @@ sqlfunction_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
/*
* sqlfunction_receive --- receive one tuple
*/
-static void
+static bool
sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
{
DR_sqlfunction *myState = (DR_sqlfunction *) self;
@@ -1913,6 +1913,8 @@ sqlfunction_receive(TupleTableSlot *slot, DestReceiver *self)
/* Store the filtered tuple into the tuplestore */
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
diff --git a/src/backend/executor/instrument.c b/src/backend/executor/instrument.c
index f5351eb..639eb04 100644
--- a/src/backend/executor/instrument.c
+++ b/src/backend/executor/instrument.c
@@ -19,9 +19,6 @@
BufferUsage pgBufferUsage;
-static void BufferUsageAccumDiff(BufferUsage *dst,
- const BufferUsage *add, const BufferUsage *sub);
-
/* Allocate new instrumentation structure(s) */
Instrumentation *
@@ -127,8 +124,29 @@ InstrEndLoop(Instrumentation *instr)
instr->tuplecount = 0;
}
+/*
+ * Aggregate the instrumentation information. This is used to aggregate
+ * the information of worker backends. We only need to sum the buffer
+ * usage and tuple count statistics as for other timing related statistics
+ * it is sufficient to have the master backends information.
+ */
+void
+InstrAggNode(Instrumentation *instr1, Instrumentation *instr2)
+{
+ /* count the returned tuples */
+ instr1->tuplecount += instr2->tuplecount;
+
+ instr1->nfiltered1 += instr2->nfiltered1;
+ instr1->nfiltered2 += instr2->nfiltered2;
+
+ /* Add delta of buffer usage since entry to node's totals */
+ if (instr1->need_bufusage)
+ BufferUsageAdd(&instr1->bufusage, &instr2->bufusage);
+
+}
+
/* dst += add - sub */
-static void
+void
BufferUsageAccumDiff(BufferUsage *dst,
const BufferUsage *add,
const BufferUsage *sub)
@@ -148,3 +166,21 @@ BufferUsageAccumDiff(BufferUsage *dst,
INSTR_TIME_ACCUM_DIFF(dst->blk_write_time,
add->blk_write_time, sub->blk_write_time);
}
+
+/* dst += add */
+void
+BufferUsageAdd(BufferUsage *dst, const BufferUsage *add)
+{
+ dst->shared_blks_hit += add->shared_blks_hit;
+ dst->shared_blks_read += add->shared_blks_read;
+ dst->shared_blks_dirtied += add->shared_blks_dirtied;
+ dst->shared_blks_written += add->shared_blks_written;
+ dst->local_blks_hit += add->local_blks_hit;
+ dst->local_blks_read += add->local_blks_read;
+ dst->local_blks_dirtied += add->local_blks_dirtied;
+ dst->local_blks_written += add->local_blks_written;
+ dst->temp_blks_read += add->temp_blks_read;
+ dst->temp_blks_written += add->temp_blks_written;
+ INSTR_TIME_ADD(dst->blk_read_time, add->blk_read_time);
+ INSTR_TIME_ADD(dst->blk_write_time, add->blk_write_time);
+}
diff --git a/src/backend/executor/nodeFunnel.c b/src/backend/executor/nodeFunnel.c
new file mode 100644
index 0000000..5266cb8
--- /dev/null
+++ b/src/backend/executor/nodeFunnel.c
@@ -0,0 +1,436 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.c
+ * Support routines for scanning a relation via multiple workers.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodeFunnel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecFunnel scans a relation using worker backends.
+ * ExecInitFunnel creates and initializes a funnel node.
+ * ExecEndFunnel releases any storage allocated.
+ * ExecReScanFunnel Re-initialize the workers and rescans a relation via them.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodeFunnel.h"
+#include "executor/nodeSubplan.h"
+#include "utils/rel.h"
+
+
+static TupleTableSlot *funnel_getnext(FunnelState *funnelstate);
+static void ExecAccumulateInstInfo(FunnelState *node);
+static void ExecAccumulateBufUsageInfo(FunnelState *node);
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * InitFunnel
+ *
+ * Set up parallel state information
+ * ----------------------------------------------------------------
+ */
+static void
+InitFunnel(FunnelState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((SeqScan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitFunnel
+ * ----------------------------------------------------------------
+ */
+FunnelState *
+ExecInitFunnel(Funnel *node, EState *estate, int eflags)
+{
+ FunnelState *funnelstate;
+
+ /* Funnel node doesn't have innerPlan node. */
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ funnelstate = makeNode(FunnelState);
+ funnelstate->ss.ps.plan = (Plan *) node;
+ funnelstate->ss.ps.state = estate;
+ funnelstate->fs_workersReady = false;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &funnelstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ funnelstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.targetlist,
+ (PlanState *) funnelstate);
+ funnelstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->scan.plan.qual,
+ (PlanState *) funnelstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &funnelstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &funnelstate->ss);
+
+ InitFunnel(funnelstate, estate, eflags);
+
+ /*
+ * now initialize outer plan
+ */
+ outerPlanState(funnelstate) = ExecInitNode(outerPlan(node), estate, eflags);
+
+
+ funnelstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&funnelstate->ss.ps);
+ ExecAssignScanProjectionInfo(&funnelstate->ss);
+
+ return funnelstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecFunnel(node)
+ *
+ * Scans the relation via multiple workers and returns
+ * the next qualifying tuple.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecFunnel(FunnelState *node)
+{
+ int i;
+ TupleTableSlot *slot;
+
+ /*
+ * Initialize the parallel context and workers on first execution.
+ * We do this on first execution rather than during node initialization,
+ * as it needs to allocate large dynamic segement, so it is better to
+ * do if it is really needed.
+ */
+ if (!node->pcxt)
+ {
+ EState *estate = node->ss.ps.state;
+ ExprContext *econtext = node->ss.ps.ps_ExprContext;
+ bool any_worker_launched = false;
+ List *serialized_param_exec;
+
+ /*
+ * 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);
+
+ /* Initialize the workers required to execute funnel node. */
+ InitializeParallelWorkers(node->ss.ps.lefttree,
+ serialized_param_exec,
+ estate,
+ &node->inst_options_space,
+ &node->buffer_usage_space,
+ &node->responseq,
+ &node->pcxt,
+ ((Funnel *)(node->ss.ps.plan))->num_workers);
+
+ outerPlanState(node)->toc = node->pcxt->toc;
+
+ /*
+ * Register backend workers. If the required number of workers are
+ * not available then we perform the scan with available workers and
+ * if there are no more workers available, then the funnel node will
+ * just scan locally.
+ */
+ LaunchParallelWorkers(node->pcxt);
+
+ node->funnel = CreateTupleQueueFunnel();
+
+ for (i = 0; i < node->pcxt->nworkers; ++i)
+ {
+ if (node->pcxt->worker[i].bgwhandle)
+ {
+ shm_mq_set_handle((node->responseq)[i], node->pcxt->worker[i].bgwhandle);
+ RegisterTupleQueueOnFunnel(node->funnel, (node->responseq)[i]);
+ any_worker_launched = true;
+ }
+ }
+
+ if (any_worker_launched)
+ node->fs_workersReady = true;
+ }
+
+ slot = funnel_getnext(node);
+
+ if (TupIsNull(slot))
+ {
+
+ /*
+ * Destroy the parallel context once we complete fetching all
+ * the tuples, this will ensure that if in the same statement
+ * we need to have Funnel node for multiple parts of statement,
+ * it won't accumulate lot of dsm segments and workers can be made
+ * available to use by other parts of statement.
+ */
+ FinishParallelSetupAndAccumStats(node);
+ }
+ return slot;
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndFunnel
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndFunnel(FunnelState *node)
+{
+ Relation relation;
+
+ relation = node->ss.ss_currentRelation;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+
+ ExecEndNode(outerPlanState(node));
+
+ FinishParallelSetupAndAccumStats(node);
+}
+
+/*
+ * funnel_getnext
+ *
+ * Get the next tuple from shared memory queue. This function
+ * is reponsible for fetching tuples from all the queues associated
+ * with worker backends used in funnel scan and if there is no
+ * data available from queues or no worker is available, it does
+ * fetch the data from local node.
+ */
+TupleTableSlot *
+funnel_getnext(FunnelState *funnelstate)
+{
+ PlanState *outerPlan;
+ TupleTableSlot *outerTupleSlot;
+ TupleTableSlot *slot;
+ HeapTuple tup;
+
+ if (funnelstate->ss.ps.ps_ProjInfo)
+ slot = funnelstate->ss.ps.ps_ProjInfo->pi_slot;
+ else
+ slot = funnelstate->ss.ss_ScanTupleSlot;
+
+ while ((!funnelstate->all_workers_done && funnelstate->fs_workersReady) ||
+ !funnelstate->local_scan_done)
+ {
+ if (!funnelstate->all_workers_done && funnelstate->fs_workersReady)
+ {
+ /* wait only if local scan is done */
+ tup = TupleQueueFunnelNext(funnelstate->funnel,
+ !funnelstate->local_scan_done,
+ &funnelstate->all_workers_done);
+
+ if (HeapTupleIsValid(tup))
+ {
+ ExecStoreTuple(tup, /* tuple to store */
+ slot, /* slot to store in */
+ InvalidBuffer, /* buffer associated with this
+ * tuple */
+ true); /* pfree this pointer if not from heap */
+
+ return slot;
+ }
+ }
+ if (!funnelstate->local_scan_done)
+ {
+ outerPlan = outerPlanState(funnelstate);
+
+ outerTupleSlot = ExecProcNode(outerPlan);
+
+ if (!TupIsNull(outerTupleSlot))
+ return outerTupleSlot;
+
+ funnelstate->local_scan_done = true;
+ }
+ }
+
+ return ExecClearTuple(slot);
+}
+
+/* ----------------------------------------------------------------
+ * FinishParallelSetupAndAccumStats
+ *
+ * Destroy the setup for parallel workers. Collect all the
+ * stats after workers are stopped, else some work done by
+ * workers won't be accounted.
+ * ----------------------------------------------------------------
+ */
+void
+FinishParallelSetupAndAccumStats(FunnelState *node)
+{
+ if (node->pcxt)
+ {
+ /*
+ * Ensure all workers have finished before destroying the parallel
+ * context to ensure a clean exit.
+ */
+ if (node->fs_workersReady)
+ {
+ TupleQueueFunnelShutdown(node->funnel);
+ WaitForParallelWorkersToFinish(node->pcxt);
+ }
+
+ /* destroy the tuple queue */
+ DestroyTupleQueueFunnel(node->funnel);
+ node->funnel = NULL;
+
+ /*
+ * Aggregate the buffer usage stats from all workers. This is
+ * required by external modules like pg_stat_statements.
+ */
+ ExecAccumulateBufUsageInfo(node);
+
+ /*
+ * Aggregate instrumentation information of all the backend
+ * workers for Funnel node. This has to be done before we
+ * destroy the parallel context.
+ */
+ if (node->ss.ps.state->es_instrument)
+ ExecAccumulateInstInfo(node);
+
+ /* destroy parallel context. */
+ DestroyParallelContext(node->pcxt);
+ node->pcxt = NULL;
+
+ node->fs_workersReady = false;
+ node->all_workers_done = false;
+ node->local_scan_done = false;
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateInstInfo
+ *
+ * Accumulate instrumentation information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateInstInfo(FunnelState *node)
+{
+ int i;
+ Instrumentation *instrument_worker;
+ int nworkers;
+ char *inst_info_workers;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ inst_info_workers = node->inst_options_space;
+ for (i = 0; i < nworkers; i++)
+ {
+ instrument_worker = (Instrumentation *)(inst_info_workers + (i * sizeof(Instrumentation)));
+ InstrAggNode(node->ss.ps.instrument, instrument_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * ExecAccumulateBufUsageInfo
+ *
+ * Accumulate buffer usage information of all the workers
+ * ----------------------------------------------------------------
+ */
+void ExecAccumulateBufUsageInfo(FunnelState *node)
+{
+ int i;
+ int nworkers;
+ BufferUsage *buffer_usage_worker;
+ char *buffer_usage;
+
+ if (node->pcxt)
+ {
+ nworkers = node->pcxt->nworkers;
+ buffer_usage = node->buffer_usage_space;
+
+ for (i = 0; i < nworkers; i++)
+ {
+ buffer_usage_worker = (BufferUsage *)(buffer_usage + (i * sizeof(BufferUsage)));
+ BufferUsageAdd(&pgBufferUsage, buffer_usage_worker);
+ }
+ }
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanFunnel
+ *
+ * Re-initialize the workers and rescans a relation via them.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanFunnel(FunnelState *node)
+{
+ /*
+ * 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);
+
+ ExecReScan(node->ss.ps.lefttree);
+}
diff --git a/src/backend/executor/nodeNestloop.c b/src/backend/executor/nodeNestloop.c
index e66bcda..c447062 100644
--- a/src/backend/executor/nodeNestloop.c
+++ b/src/backend/executor/nodeNestloop.c
@@ -144,6 +144,7 @@ ExecNestLoop(NestLoopState *node)
{
NestLoopParam *nlp = (NestLoopParam *) lfirst(lc);
int paramno = nlp->paramno;
+ TupleDesc tdesc = outerTupleSlot->tts_tupleDescriptor;
ParamExecData *prm;
prm = &(econtext->ecxt_param_exec_vals[paramno]);
@@ -154,6 +155,7 @@ ExecNestLoop(NestLoopState *node)
prm->value = slot_getattr(outerTupleSlot,
nlp->paramval->varattno,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[nlp->paramval->varattno-1]->atttypid;
/* Flag parameter value as changed */
innerPlan->chgParam = bms_add_member(innerPlan->chgParam,
paramno);
diff --git a/src/backend/executor/nodeSubplan.c b/src/backend/executor/nodeSubplan.c
index 9eb4d63..6afd55a 100644
--- a/src/backend/executor/nodeSubplan.c
+++ b/src/backend/executor/nodeSubplan.c
@@ -30,11 +30,14 @@
#include <math.h>
#include "access/htup_details.h"
+#include "catalog/pg_type.h"
#include "executor/executor.h"
#include "executor/nodeSubplan.h"
#include "nodes/makefuncs.h"
+#include "nodes/nodeFuncs.h"
#include "optimizer/clauses.h"
#include "utils/array.h"
+#include "utils/datum.h"
#include "utils/lsyscache.h"
#include "utils/memutils.h"
@@ -281,12 +284,14 @@ ExecScanSubPlan(SubPlanState *node,
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -399,6 +404,7 @@ ExecScanSubPlan(SubPlanState *node,
prmdata = &(econtext->ecxt_param_exec_vals[paramid]);
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col, &(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
@@ -551,6 +557,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
!TupIsNull(slot);
slot = ExecProcNode(planstate))
{
+ TupleDesc tdesc = slot->tts_tupleDescriptor;
int col = 1;
ListCell *plst;
bool isnew;
@@ -568,6 +575,7 @@ buildSubPlanHash(SubPlanState *node, ExprContext *econtext)
Assert(prmdata->execPlan == NULL);
prmdata->value = slot_getattr(slot, col,
&(prmdata->isnull));
+ prmdata->ptype = tdesc->attrs[col-1]->atttypid;
col++;
}
slot = ExecProject(node->projRight, NULL);
@@ -954,6 +962,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
ListCell *l;
bool found = false;
ArrayBuildStateAny *astate = NULL;
+ Oid ptype;
if (subLinkType == ANY_SUBLINK ||
subLinkType == ALL_SUBLINK)
@@ -961,6 +970,8 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
if (subLinkType == CTE_SUBLINK)
elog(ERROR, "CTE subplans should not be executed via ExecSetParamPlan");
+ ptype = exprType((Node *) node->xprstate.expr);
+
/* Initialize ArrayBuildStateAny in caller's context, if needed */
if (subLinkType == ARRAY_SUBLINK)
astate = initArrayResultAny(subplan->firstColType,
@@ -983,12 +994,14 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
forboth(l, subplan->parParam, pvar, node->args)
{
int paramid = lfirst_int(l);
+ ExprState *exprstate = (ExprState *) lfirst(pvar);
ParamExecData *prm = &(econtext->ecxt_param_exec_vals[paramid]);
- prm->value = ExecEvalExprSwitchContext((ExprState *) lfirst(pvar),
+ prm->value = ExecEvalExprSwitchContext(exprstate,
econtext,
&(prm->isnull),
NULL);
+ prm->ptype = exprType((Node *) exprstate->expr);
planstate->chgParam = bms_add_member(planstate->chgParam, paramid);
}
@@ -1011,6 +1024,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(true);
+ prm->ptype = ptype;
prm->isnull = false;
found = true;
break;
@@ -1062,6 +1076,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = heap_getattr(node->curTuple, i, tdesc,
&(prm->isnull));
+ prm->ptype = tdesc->attrs[i-1]->atttypid;
i++;
}
}
@@ -1084,6 +1099,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
true);
prm->execPlan = NULL;
prm->value = node->curArray;
+ prm->ptype = ptype;
prm->isnull = false;
}
else if (!found)
@@ -1096,6 +1112,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = BoolGetDatum(false);
+ prm->ptype = ptype;
prm->isnull = false;
}
else
@@ -1108,6 +1125,7 @@ ExecSetParamPlan(SubPlanState *node, ExprContext *econtext)
prm->execPlan = NULL;
prm->value = (Datum) 0;
+ prm->ptype = VOIDOID;
prm->isnull = true;
}
}
@@ -1238,3 +1256,47 @@ ExecAlternativeSubPlan(AlternativeSubPlanState *node,
isNull,
isDone);
}
+
+/*
+ * ExecAndFormSerializeParamExec
+ *
+ * Execute the subplan stored in PARAM_EXEC param if it is not executed
+ * till now and form the serialized structure required for passing to
+ * worker backend.
+ */
+List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params)
+{
+ List *lparam = NIL;
+ SerializedParamExecData *sparamdata;
+ ParamExecData *prm;
+ int paramid;
+
+ paramid = -1;
+ while ((paramid = bms_next_member(params, paramid)) >= 0)
+ {
+ /*
+ * PARAM_EXEC params (internal executor parameters) are stored in the
+ * ecxt_param_exec_vals array, and can be accessed by array index.
+ */
+ sparamdata = palloc0(sizeof(SerializedParamExecData));
+
+ prm = &(econtext->ecxt_param_exec_vals[paramid]);
+ if (prm->execPlan != NULL)
+ {
+ /* Parameter not evaluated yet, so go do it */
+ ExecSetParamPlan(prm->execPlan, econtext);
+ /* ExecSetParamPlan should have processed this param... */
+ Assert(prm->execPlan == NULL);
+ }
+
+ sparamdata->paramid = paramid;
+ sparamdata->ptype = prm->ptype;
+ sparamdata->value = prm->value;
+ sparamdata->isnull = prm->isnull;
+
+ lparam = lappend(lparam, sparamdata);
+ }
+
+ return lparam;
+}
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index d544ad9..d8ca074 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -1774,7 +1774,7 @@ spi_dest_startup(DestReceiver *self, int operation, TupleDesc typeinfo)
* store tuple retrieved by Executor into SPITupleTable
* of current SPI procedure
*/
-void
+bool
spi_printtup(TupleTableSlot *slot, DestReceiver *self)
{
SPITupleTable *tuptable;
@@ -1808,6 +1808,8 @@ spi_printtup(TupleTableSlot *slot, DestReceiver *self)
(tuptable->free)--;
MemoryContextSwitchTo(oldcxt);
+
+ return true;
}
/*
diff --git a/src/backend/executor/tqueue.c b/src/backend/executor/tqueue.c
new file mode 100644
index 0000000..39acda7
--- /dev/null
+++ b/src/backend/executor/tqueue.c
@@ -0,0 +1,304 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.c
+ * Use shm_mq to send & receive tuples between parallel backends
+ *
+ * A DestReceiver of type DestTupleQueue, which is a TQueueDestReciever
+ * under the hood, writes tuples from the executor to a shm_mq.
+ *
+ * A TupleQueueFunnel helps manage the process of reading tuples from
+ * one or more shm_mq objects being used as tuple queues.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/executor/tqueue.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/htup_details.h"
+#include "executor/tqueue.h"
+#include "miscadmin.h"
+
+typedef struct
+{
+ DestReceiver pub;
+ shm_mq_handle *handle;
+} TQueueDestReceiver;
+
+struct TupleQueueFunnel
+{
+ int nqueues;
+ int maxqueues;
+ int nextqueue;
+ shm_mq_handle **queue;
+};
+
+/*
+ * Receive a tuple.
+ */
+static bool
+tqueueReceiveSlot(TupleTableSlot *slot, DestReceiver *self)
+{
+ TQueueDestReceiver *tqueue = (TQueueDestReceiver *) self;
+ HeapTuple tuple;
+ shm_mq_result result;
+
+ tuple = ExecMaterializeSlot(slot);
+ result = shm_mq_send(tqueue->handle, tuple->t_len, tuple->t_data, false);
+
+ if (result == SHM_MQ_DETACHED)
+ return false;
+ else if (result != SHM_MQ_SUCCESS)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to send tuples")));
+
+ return true;
+}
+
+/*
+ * Prepare to receive tuples from executor.
+ */
+static void
+tqueueStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
+{
+ /* do nothing */
+}
+
+/*
+ * Clean up at end of an executor run
+ */
+static void
+tqueueShutdownReceiver(DestReceiver *self)
+{
+ /* do nothing */
+}
+
+/*
+ * Destroy receiver when done with it
+ */
+static void
+tqueueDestroyReceiver(DestReceiver *self)
+{
+ pfree(self);
+}
+
+/*
+ * Create a DestReceiver that writes tuples to a tuple queue.
+ */
+DestReceiver *
+CreateTupleQueueDestReceiver(void)
+{
+ TQueueDestReceiver *self;
+
+ self = (TQueueDestReceiver *) palloc0(sizeof(TQueueDestReceiver));
+
+ self->pub.receiveSlot = tqueueReceiveSlot;
+ self->pub.rStartup = tqueueStartupReceiver;
+ self->pub.rShutdown = tqueueShutdownReceiver;
+ self->pub.rDestroy = tqueueDestroyReceiver;
+ self->pub.mydest = DestTupleQueue;
+
+ /* private fields will be set by SetTupleQueueDestReceiverParams */
+
+ return (DestReceiver *) self;
+}
+
+/*
+ * Set parameters for a TupleQueueDestReceiver
+ */
+void
+SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle)
+{
+ TQueueDestReceiver *myState = (TQueueDestReceiver *) self;
+
+ myState->handle = handle;
+}
+
+/*
+ * Create a tuple queue funnel.
+ */
+TupleQueueFunnel *
+CreateTupleQueueFunnel(void)
+{
+ TupleQueueFunnel *funnel = palloc0(sizeof(TupleQueueFunnel));
+
+ funnel->maxqueues = 8;
+ funnel->queue = palloc(funnel->maxqueues * sizeof(shm_mq_handle *));
+
+ return funnel;
+}
+
+/*
+ * Detach all tuple queues that belong to funnel.
+ */
+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);
+ }
+ }
+}
+
+/*
+ * Destroy a tuple queue funnel.
+ */
+void
+DestroyTupleQueueFunnel(TupleQueueFunnel *funnel)
+{
+ if (funnel)
+ {
+ pfree(funnel->queue);
+ pfree(funnel);
+ }
+}
+
+/*
+ * Remember the shared memory queue handle in funnel.
+ */
+void
+RegisterTupleQueueOnFunnel(TupleQueueFunnel *funnel, shm_mq_handle *handle)
+{
+ if (funnel->nqueues < funnel->maxqueues)
+ {
+ funnel->queue[funnel->nqueues++] = handle;
+ return;
+ }
+
+ if (funnel->nqueues >= funnel->maxqueues)
+ {
+ int newsize = funnel->nqueues * 2;
+
+ Assert(funnel->nqueues == funnel->maxqueues);
+
+ funnel->queue = repalloc(funnel->queue,
+ newsize * sizeof(shm_mq_handle *));
+ funnel->maxqueues = newsize;
+ }
+
+ funnel->queue[funnel->nqueues++] = handle;
+}
+
+/*
+ * Fetch a tuple from a tuple queue funnel.
+ *
+ * We try to read from the queues in round-robin fashion so as to avoid
+ * the situation where some workers get their tuples read expediently while
+ * others are barely ever serviced.
+ *
+ * Even when nowait = false, we read from the individual queues in
+ * non-blocking mode. Even when shm_mq_receive() returns SHM_MQ_WOULD_BLOCK,
+ * it can still accumulate bytes from a partially-read message, so doing it
+ * this way should outperform doing a blocking read on each queue in turn.
+ *
+ * The return value is NULL if there are no remaining queues or if
+ * nowait = true and no queue returned a tuple without blocking. *done, if
+ * not NULL, is set to true when there are no remaining queues and false in
+ * any other case.
+ */
+HeapTuple
+TupleQueueFunnelNext(TupleQueueFunnel *funnel, bool nowait, bool *done)
+{
+ int waitpos = funnel->nextqueue;
+
+ /* Corner case: called before adding any queues, or after all are gone. */
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ if (done != NULL)
+ *done = false;
+
+ for (;;)
+ {
+ shm_mq_handle *mqh = funnel->queue[funnel->nextqueue];
+ shm_mq_result result;
+ Size nbytes;
+ void *data;
+
+ /* Attempt to read a message. */
+ result = shm_mq_receive(mqh, &nbytes, &data, true);
+
+ /*
+ * Normally, we advance funnel->nextqueue to the next queue at this
+ * point, but if we're pointing to a queue that we've just discovered
+ * is detached, then forget that queue and leave the pointer where it
+ * is until the number of remaining queues fall below that pointer and
+ * at that point make the pointer point to the first queue.
+ */
+ if (result != SHM_MQ_DETACHED)
+ funnel->nextqueue = (funnel->nextqueue + 1) % funnel->nqueues;
+ else
+ {
+ --funnel->nqueues;
+ if (funnel->nqueues == 0)
+ {
+ if (done != NULL)
+ *done = true;
+ return NULL;
+ }
+
+ memmove(&funnel->queue[funnel->nextqueue],
+ &funnel->queue[funnel->nextqueue + 1],
+ sizeof(shm_mq_handle *)
+ * (funnel->nqueues - funnel->nextqueue));
+
+ if (funnel->nextqueue >= funnel->nqueues)
+ funnel->nextqueue = 0;
+
+ if (funnel->nextqueue < waitpos)
+ --waitpos;
+
+ continue;
+ }
+
+ /* If we got a message, return it. */
+ if (result == SHM_MQ_SUCCESS)
+ {
+ HeapTupleData htup;
+
+ /*
+ * The tuple data we just read from the queue is only valid
+ * until we again attempt to read from it. Copy the tuple into
+ * a single palloc'd chunk as callers will expect.
+ */
+ ItemPointerSetInvalid(&htup.t_self);
+ htup.t_tableOid = InvalidOid;
+ htup.t_len = nbytes;
+ htup.t_data = data;
+ return heap_copytuple(&htup);
+ }
+
+ /*
+ * If we've visited all of the queues, then we should either give up
+ * and return NULL (if we're in non-blocking mode) or wait for the
+ * process latch to be set (otherwise).
+ */
+ if (funnel->nextqueue == waitpos)
+ {
+ if (nowait)
+ return NULL;
+ WaitLatch(MyLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(MyLatch);
+ }
+ }
+}
diff --git a/src/backend/executor/tstoreReceiver.c b/src/backend/executor/tstoreReceiver.c
index c1fdeb7..b0862ae 100644
--- a/src/backend/executor/tstoreReceiver.c
+++ b/src/backend/executor/tstoreReceiver.c
@@ -37,8 +37,8 @@ typedef struct
} TStoreState;
-static void tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
-static void tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self);
+static bool tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self);
/*
@@ -90,19 +90,21 @@ tstoreStartupReceiver(DestReceiver *self, int operation, TupleDesc typeinfo)
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the easy case where we don't have to detoast.
*/
-static void
+static bool
tstoreReceiveSlot_notoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
tuplestore_puttupleslot(myState->tstore, slot);
+
+ return true;
}
/*
* Receive a tuple from the executor and store it in the tuplestore.
* This is for the case where we have to detoast any toasted values.
*/
-static void
+static bool
tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
{
TStoreState *myState = (TStoreState *) self;
@@ -152,6 +154,8 @@ tstoreReceiveSlot_detoast(TupleTableSlot *slot, DestReceiver *self)
/* And release any temporary detoasted values */
for (i = 0; i < nfree; i++)
pfree(DatumGetPointer(myState->tofree[i]));
+
+ return true;
}
/*
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 1d3dd22..11d8191 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -382,6 +382,27 @@ _copySampleScan(const SampleScan *from)
}
/*
+ * _copyFunnel
+ */
+static Funnel *
+_copyFunnel(const Funnel *from)
+{
+ Funnel *newnode = makeNode(Funnel);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ /*
+ * copy remainder of node
+ */
+ COPY_SCALAR_FIELD(num_workers);
+
+ return newnode;
+}
+
+/*
* _copyIndexScan
*/
static IndexScan *
@@ -4239,6 +4260,9 @@ copyObject(const void *from)
case T_SampleScan:
retval = _copySampleScan(from);
break;
+ case T_Funnel:
+ retval = _copyFunnel(from);
+ break;
case T_IndexScan:
retval = _copyIndexScan(from);
break;
diff --git a/src/backend/nodes/nodeFuncs.c b/src/backend/nodes/nodeFuncs.c
index c517dfd..0cf34db 100644
--- a/src/backend/nodes/nodeFuncs.c
+++ b/src/backend/nodes/nodeFuncs.c
@@ -3412,3 +3412,25 @@ raw_expression_tree_walker(Node *node,
}
return false;
}
+
+/*
+ * planstate_tree_walker
+ *
+ * This routine will invoke walker on the node passed. This is a useful
+ * way of starting the recursion when the walker's normal change of state
+ * is not appropriate for the outermost PlanState node.
+ */
+bool
+planstate_tree_walker(Node *node,
+ ParallelContext *pcxt,
+ bool (*walker) (),
+ void *context)
+{
+ if (node == NULL)
+ return false;
+
+ /* Guard against stack overflow due to overly complex plan */
+ check_stack_depth();
+
+ return walker(node, pcxt, context);
+}
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 152e715..232b950 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -456,6 +456,16 @@ _outSampleScan(StringInfo str, const SampleScan *node)
}
static void
+_outFunnel(StringInfo str, const Funnel *node)
+{
+ WRITE_NODE_TYPE("FUNNEL");
+
+ _outScanInfo(str, (const Scan *) node);
+
+ WRITE_UINT_FIELD(num_workers);
+}
+
+static void
_outIndexScan(StringInfo str, const IndexScan *node)
{
WRITE_NODE_TYPE("INDEXSCAN");
@@ -3005,6 +3015,9 @@ _outNode(StringInfo str, const void *obj)
case T_SampleScan:
_outSampleScan(str, obj);
break;
+ case T_Funnel:
+ _outFunnel(str, obj);
+ break;
case T_IndexScan:
_outIndexScan(str, obj);
break;
diff --git a/src/backend/nodes/params.c b/src/backend/nodes/params.c
index fb803f8..e81afbd 100644
--- a/src/backend/nodes/params.c
+++ b/src/backend/nodes/params.c
@@ -16,9 +16,22 @@
#include "postgres.h"
#include "nodes/params.h"
+#include "storage/shmem.h"
#include "utils/datum.h"
#include "utils/lsyscache.h"
+/*
+ * for each bind parameter, pass this structure followed by value
+ * except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExternData
+{
+ Datum value; /*pass-by-val are directly stored */
+ Size length; /* length of parameter value */
+ bool isnull; /* is it NULL? */
+ uint16 pflags; /* flag bits, same as in original Param */
+ Oid ptype; /* parameter's datatype, or 0 */
+} SerializedParamExternData;
/*
* Copy a ParamListInfo structure.
@@ -73,3 +86,354 @@ copyParamList(ParamListInfo from)
return retval;
}
+
+/*
+ * Estimate the amount of space required to serialize the bound
+ * parameters.
+ */
+Size
+EstimateBoundParametersSpace(ParamListInfo paramInfo)
+{
+ Size size;
+ int i;
+
+ /* Add space required for saving numParams */
+ size = sizeof(int);
+
+ if (paramInfo)
+ {
+ /* Add space required for saving the param data */
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ /*
+ * for each parameter, calculate the size of fixed part
+ * of parameter (SerializedParamExternData) and length of
+ * parameter value.
+ */
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size length;
+
+ length = sizeof(SerializedParamExternData);
+
+ oprm = ¶mInfo->params[i];
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || oprm->isnull))
+ {
+ length += datumGetSize(oprm->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the bind parameters into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateBoundParametersSpace.
+ */
+void
+SerializeBoundParams(ParamListInfo paramInfo, Size maxsize, char *start_address)
+{
+ char *curptr;
+ SerializedParamExternData *retval;
+ int i;
+
+ /*
+ * First, we store the number of bind parameters, if there is
+ * no bind parameter then no need to store any more information.
+ */
+ if (paramInfo && paramInfo->numParams > 0)
+ * (int *) start_address = paramInfo->numParams;
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+ curptr = start_address + sizeof(int);
+
+
+ for (i = 0; i < paramInfo->numParams; i++)
+ {
+ ParamExternData *oprm;
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+
+ Assert (curptr <= start_address + maxsize);
+ retval = (SerializedParamExternData*) curptr;
+ oprm = ¶mInfo->params[i];
+
+ retval->isnull = oprm->isnull;
+ retval->pflags = oprm->pflags;
+ retval->ptype = oprm->ptype;
+ retval->value = oprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(oprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(oprm->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(oprm->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreBoundParams
+ * Restore bind parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+ParamListInfo
+RestoreBoundParams(char *start_address)
+{
+ ParamListInfo retval;
+ Size size;
+ int num_params,i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ size = offsetof(ParamListInfoData, params) +
+ num_params * sizeof(ParamExternData);
+ retval = (ParamListInfo) palloc(size);
+ retval->paramFetch = NULL;
+ retval->paramFetchArg = NULL;
+ retval->parserSetup = NULL;
+ retval->parserSetupArg = NULL;
+ retval->numParams = num_params;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExternData *nprm;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExternData *) curptr;
+
+ /* copy the parameter info */
+ retval->params[i].isnull = nprm->isnull;
+ retval->params[i].pflags = nprm->pflags;
+ retval->params[i].ptype = nprm->ptype;
+ retval->params[i].value = nprm->value;
+
+ curptr = curptr + sizeof(SerializedParamExternData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ retval->params[i].value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+ }
+
+ return retval;
+}
+
+/*
+ * Estimate the amount of space required to serialize the PARAM_EXEC
+ * parameters.
+ */
+Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals)
+{
+ Size size;
+ ListCell *lparam;
+
+ /*
+ * Add space required for saving number of PARAM_EXEC parameters
+ * that needs to be serialized.
+ */
+ size = sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size length;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ length = sizeof(SerializedParamExecData);
+
+ get_typlenbyval(param_val->ptype, &typLen, &typByVal);
+
+ /*
+ * pass-by-value parameters are directly stored in
+ * SerializedParamExternData, so no need of additional
+ * space for them.
+ */
+ if (!(typByVal || param_val->isnull))
+ {
+ length += datumGetSize(param_val->value, typByVal, typLen);
+ size = add_size(size, length);
+
+ /* Allow space for terminating zero-byte */
+ size = add_size(size, 1);
+ }
+ else
+ size = add_size(size, length);
+ }
+
+ return size;
+}
+
+/*
+ * Serialize the PARAM_EXEC parameters into the memory, beginning at
+ * start_address. maxsize should be at least as large as the value
+ * returned by EstimateExecParametersSpace.
+ */
+void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address)
+{
+ char *curptr;
+ SerializedParamExecData *retval;
+ ListCell *lparam;
+
+ /*
+ * First, we store the number of PARAM_EXEC parameters that needs to
+ * be serialized.
+ */
+ if (serialized_param_exec_vals)
+ * (int *) start_address = list_length(serialized_param_exec_vals);
+ else
+ {
+ * (int *) start_address = 0;
+ return;
+ }
+
+ curptr = start_address + sizeof(int);
+
+ foreach(lparam, serialized_param_exec_vals)
+ {
+ int16 typLen;
+ bool typByVal;
+ Size datumlength, length;
+ const char *s;
+ SerializedParamExecData* param_val = (SerializedParamExecData*) lfirst(lparam);
+
+ retval = (SerializedParamExecData*) curptr;
+
+ retval->paramid = param_val->paramid;
+ retval->value = param_val->value;
+ retval->isnull = param_val->isnull;
+ retval->ptype = param_val->ptype;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (retval->isnull)
+ continue;
+
+ get_typlenbyval(retval->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ datumlength = datumGetSize(retval->value, typByVal, typLen);
+ s = (char *) DatumGetPointer(retval->value);
+ memcpy(curptr, s, datumlength);
+ length = datumlength;
+ curptr[length] = '\0';
+ retval->length = length;
+ curptr += length + 1;
+ }
+ }
+}
+
+/*
+ * RestoreExecParams
+ * Restore PARAM_EXEC parameters from the specified address.
+ *
+ * The params are palloc'd in CurrentMemoryContext.
+ */
+List *
+RestoreExecParams(char *start_address)
+{
+ List *lparamexecvals = NIL;
+ int num_params, i;
+ char *curptr;
+
+ num_params = * (int *) start_address;
+
+ if (num_params <= 0)
+ return NULL;
+
+ curptr = start_address + sizeof(int);
+
+ for (i = 0; i < num_params; i++)
+ {
+ SerializedParamExecData *nprm;
+ SerializedParamExecData *outparam;
+ char *s;
+ int16 typLen;
+ bool typByVal;
+
+ nprm = (SerializedParamExecData *) curptr;
+
+ outparam = palloc0(sizeof(SerializedParamExecData));
+
+ /* copy the parameter info */
+ outparam->isnull = nprm->isnull;
+ outparam->value = nprm->value;
+ outparam->paramid = nprm->paramid;
+
+ curptr = curptr + sizeof(SerializedParamExecData);
+
+ if (nprm->isnull)
+ continue;
+
+ get_typlenbyval(nprm->ptype, &typLen, &typByVal);
+
+ if (!typByVal)
+ {
+ s = palloc(nprm->length + 1);
+ memcpy(s, curptr, nprm->length + 1);
+ outparam->value = CStringGetDatum(s);
+
+ curptr += nprm->length + 1;
+ }
+
+ lparamexecvals = lappend(lparamexecvals, outparam);
+ }
+
+ return lparamexecvals;
+}
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index 23e0b36..e0fe8d5 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -29,6 +29,7 @@
#include <math.h>
#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
#include "nodes/readfuncs.h"
@@ -1366,6 +1367,49 @@ _readTableSampleClause(void)
READ_DONE();
}
+/*
+ * _readPlanInvalItem
+ */
+static PlanInvalItem *
+_readPlanInvalItem(void)
+{
+ READ_LOCALS(PlanInvalItem);
+
+ READ_INT_FIELD(cacheId);
+ READ_UINT_FIELD(hashValue);
+
+ READ_DONE();
+}
+
+/*
+ * _readPlannedStmt
+ */
+static PlannedStmt *
+_readPlannedStmt(void)
+{
+ READ_LOCALS(PlannedStmt);
+
+ READ_ENUM_FIELD(commandType, CmdType);
+ READ_UINT_FIELD(queryId);
+ READ_BOOL_FIELD(hasReturning);
+ READ_BOOL_FIELD(hasModifyingCTE);
+ READ_BOOL_FIELD(canSetTag);
+ READ_BOOL_FIELD(transientPlan);
+ READ_NODE_FIELD(planTree);
+ READ_NODE_FIELD(rtable);
+ READ_NODE_FIELD(resultRelations);
+ READ_NODE_FIELD(utilityStmt);
+ READ_NODE_FIELD(subplans);
+ READ_BITMAPSET_FIELD(rewindPlanIDs);
+ READ_NODE_FIELD(rowMarks);
+ READ_NODE_FIELD(relationOids);
+ READ_NODE_FIELD(invalItems);
+ READ_INT_FIELD(nParamExec);
+ READ_BOOL_FIELD(hasRowSecurity);
+ READ_BOOL_FIELD(parallelModeNeeded);
+
+ READ_DONE();
+}
/*
* parseNodeString
@@ -1505,6 +1549,10 @@ parseNodeString(void)
return_value = _readNotifyStmt();
else if (MATCH("DECLARECURSOR", 13))
return_value = _readDeclareCursorStmt();
+ else if (MATCH("PLANINVALITEM", 13))
+ return_value = _readPlanInvalItem();
+ else if (MATCH("PLANNEDSTMT", 11))
+ return_value = _readPlannedStmt();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 7069f60..78d976a 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -11,6 +11,8 @@
* cpu_tuple_cost Cost of typical CPU time to process a tuple
* cpu_index_tuple_cost Cost of typical CPU time to process an index tuple
* cpu_operator_cost Cost of CPU time to execute an operator or function
+ * cpu_tuple_comm_cost Cost of CPU time to pass a tuple from worker to master backend
+ * parallel_setup_cost Cost of setting up shared memory for parallelism
*
* We expect that the kernel will typically do some amount of read-ahead
* optimization; this in conjunction with seek costs means that seq_page_cost
@@ -102,11 +104,15 @@ double random_page_cost = DEFAULT_RANDOM_PAGE_COST;
double cpu_tuple_cost = DEFAULT_CPU_TUPLE_COST;
double cpu_index_tuple_cost = DEFAULT_CPU_INDEX_TUPLE_COST;
double cpu_operator_cost = DEFAULT_CPU_OPERATOR_COST;
+double cpu_tuple_comm_cost = DEFAULT_CPU_TUPLE_COMM_COST;
+double parallel_setup_cost = DEFAULT_PARALLEL_SETUP_COST;
int effective_cache_size = DEFAULT_EFFECTIVE_CACHE_SIZE;
Cost disable_cost = 1.0e10;
+int parallel_seqscan_degree = 0;
+
bool enable_seqscan = true;
bool enable_indexscan = true;
bool enable_indexonlyscan = true;
@@ -290,6 +296,42 @@ cost_samplescan(Path *path, PlannerInfo *root,
}
/*
+ * cost_funnel
+ * Determines and returns the cost of funnel path.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ */
+void
+cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ /* Should only be applied to base relations */
+ Assert(baserel->relid > 0);
+ Assert(baserel->rtekind == RTE_RELATION);
+
+ /* Mark the path with the correct row estimate */
+ if (param_info)
+ path->path.rows = param_info->ppi_rows;
+ else
+ path->path.rows = baserel->rows;
+
+ startup_cost = path->subpath->startup_cost;
+
+ run_cost = path->subpath->total_cost - path->subpath->startup_cost;
+
+ /* Parallel setup and communication cost. */
+ startup_cost += parallel_setup_cost;
+ run_cost += cpu_tuple_comm_cost * baserel->tuples;
+
+ path->path.startup_cost = startup_cost;
+ path->path.total_cost = (startup_cost + run_cost);
+}
+
+/*
* cost_index
* Determines and returns the cost of scanning a relation using an index.
*
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 404c6f5..68d8837 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -60,6 +60,8 @@ static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
static SampleScan *create_samplescan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Funnel *create_funnel_plan(PlannerInfo *root,
+ FunnelPath *best_path);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
List *tlist, List *scan_clauses, bool indexonly);
static BitmapHeapScan *create_bitmap_scan_plan(PlannerInfo *root,
@@ -104,6 +106,9 @@ static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
static SampleScan *make_samplescan(List *qptlist, List *qpqual, Index scanrelid,
TableSampleClause *tsc);
+static Funnel *make_funnel(List *qptlist, List *qpqual,
+ Index scanrelid, int nworkers,
+ Plan *subplan);
static IndexScan *make_indexscan(List *qptlist, List *qpqual, Index scanrelid,
Oid indexid, List *indexqual, List *indexqualorig,
List *indexorderby, List *indexorderbyorig,
@@ -273,6 +278,10 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
plan = create_unique_plan(root,
(UniquePath *) best_path);
break;
+ case T_Funnel:
+ plan = (Plan *) create_funnel_plan(root,
+ (FunnelPath *) best_path);
+ break;
default:
elog(ERROR, "unrecognized node type: %d",
(int) best_path->pathtype);
@@ -560,6 +569,7 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -1194,6 +1204,67 @@ create_samplescan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_funnel_plan
+ *
+ * Returns a funnel plan for the base relation scanned by
+ * 'best_path'.
+ */
+static Funnel *
+create_funnel_plan(PlannerInfo *root, FunnelPath *best_path)
+{
+ Funnel *funnel_plan;
+ Plan *subplan;
+ List *tlist;
+ RelOptInfo *rel = best_path->path.parent;
+ Index scan_relid = best_path->path.parent->relid;
+
+ /*
+ * For table scans, rather than using the relation targetlist (which is
+ * only those Vars actually needed by the query), we prefer to generate a
+ * tlist containing all Vars in order. This will allow the executor to
+ * optimize away projection of the table tuples, if possible. (Note that
+ * planner.c may replace the tlist we generate here, forcing projection to
+ * occur.)
+ */
+ if (use_physical_tlist(root, rel))
+ {
+ tlist = build_physical_tlist(root, rel);
+ /* if fail because of dropped cols, use regular method */
+ if (tlist == NIL)
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+ else
+ {
+ tlist = build_path_tlist(root, &best_path->path);
+ }
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->path.parent->rtekind == RTE_RELATION);
+
+ subplan = create_plan_recurse(root, best_path->subpath);
+
+ /*
+ * quals for subplan and top level plan are same
+ * as either all the quals are pushed to subplan
+ * (partialseqscan plan) or parallel plan won't be
+ * choosen.
+ */
+ funnel_plan = make_funnel(tlist,
+ subplan->qual,
+ scan_relid,
+ best_path->num_workers,
+ subplan);
+
+ copy_path_costsize(&funnel_plan->scan.plan, &best_path->path);
+
+ /* use parallel mode for parallel plans. */
+ root->glob->parallelModeNeeded = true;
+
+ return funnel_plan;
+}
+
+/*
* create_indexscan_plan
* Returns an indexscan plan for the base relation scanned by 'best_path'
* with restriction clauses 'scan_clauses' and targetlist 'tlist'.
@@ -3462,6 +3533,27 @@ make_samplescan(List *qptlist,
return node;
}
+static Funnel *
+make_funnel(List *qptlist,
+ List *qpqual,
+ Index scanrelid,
+ int nworkers,
+ Plan *subplan)
+{
+ Funnel *node = makeNode(Funnel);
+ Plan *plan = &node->scan.plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = subplan;
+ plan->righttree = NULL;
+ node->scan.scanrelid = scanrelid;
+ node->num_workers = nworkers;
+
+ return node;
+}
+
static IndexScan *
make_indexscan(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/planner.c b/src/backend/optimizer/plan/planner.c
index 2467570..11f095e 100644
--- a/src/backend/optimizer/plan/planner.c
+++ b/src/backend/optimizer/plan/planner.c
@@ -312,6 +312,52 @@ standard_planner(Query *parse, int cursorOptions, ParamListInfo boundParams)
return result;
}
+PlannedStmt *
+create_parallel_worker_plannedstmt(Plan *plan,
+ List *rangetable,
+ int num_exec_params)
+{
+ PlannedStmt *result;
+ ListCell *tlist;
+
+ /*
+ * Avoid removing junk entries in worker as those are
+ * required by upper nodes in master backend.
+ */
+ foreach(tlist, plan->targetlist)
+ {
+ TargetEntry *tle = (TargetEntry *) lfirst(tlist);
+
+ tle->resjunk = false;
+ }
+
+ /* build the PlannedStmt result */
+ result = makeNode(PlannedStmt);
+
+ result->commandType = CMD_SELECT;
+ result->queryId = 0;
+ result->hasReturning = 0;
+ result->hasModifyingCTE = 0;
+ result->canSetTag = 1;
+ result->transientPlan = 0;
+ result->planTree = plan;
+ result->rtable = rangetable;
+ result->resultRelations = NIL;
+ result->utilityStmt = NULL;
+ result->subplans = NIL;
+ result->rewindPlanIDs = NULL;
+ result->rowMarks = NIL;
+ result->nParamExec = num_exec_params;
+ /*
+ * Don't bother to set parameters used for invalidation as
+ * worker backend plans are not saved, so can't be invalidated.
+ */
+ result->relationOids = NIL;
+ result->invalItems = NIL;
+ result->hasRowSecurity = false;
+
+ return result;
+}
/*--------------------
* subquery_planner
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index ee8710d..12f6635 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -465,6 +465,25 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
fix_scan_expr(root, (Node *) splan->tablesample, rtoffset);
}
break;
+ case T_Funnel:
+ {
+ Funnel *splan = (Funnel *) plan;
+
+ /*
+ * target list for leftree of funnel plan should be same as for
+ * funnel scan as both nodes need to produce same projection.
+ * We don't want to do this assignment after fixing references
+ * as that will be done separately for lefttree node.
+ */
+ splan->scan.plan.lefttree->targetlist = splan->scan.plan.targetlist;
+
+ splan->scan.scanrelid += rtoffset;
+ splan->scan.plan.targetlist =
+ fix_scan_list(root, splan->scan.plan.targetlist, rtoffset);
+ splan->scan.plan.qual =
+ fix_scan_list(root, splan->scan.plan.qual, rtoffset);
+ }
+ break;
case T_IndexScan:
{
IndexScan *splan = (IndexScan *) plan;
@@ -2265,6 +2284,40 @@ fix_opfuncids_walker(Node *node, void *context)
}
/*
+ * fix_node_funcids
+ * Set the opfuncid (procedure OID) in an OpExpr node,
+ * for plan tree.
+ *
+ * We need it mainly to fix the opfuncid in nodes of plantree
+ * after reading the planned statement by worker backend.
+ * Currently the support of nodes that could be executed by
+ * worker backend are limited, we can enhance this API based
+ * on it's usage in future.
+ */
+void
+fix_node_funcids(Plan *node)
+{
+ /*
+ * do nothing when we get to the end of a leaf on tree.
+ */
+ if (node == NULL)
+ return;
+
+ fix_opfuncids((Node*) node->qual);
+ fix_opfuncids((Node*) node->targetlist);
+
+ switch (nodeTag(node))
+ {
+ default:
+ elog(ERROR, "unrecognized node type: %d", (int) nodeTag(node));
+ break;
+ }
+
+ fix_node_funcids(node->lefttree);
+ fix_node_funcids(node->righttree);
+}
+
+/*
* set_opfuncid
* Set the opfuncid (procedure OID) in an OpExpr node,
* if it hasn't been set already.
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index d0bc412..073a7f5 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2243,6 +2243,10 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
context.paramids = bms_add_members(context.paramids, scan_params);
break;
+ case T_Funnel:
+ context.paramids = bms_add_members(context.paramids, scan_params);
+ break;
+
case T_IndexScan:
finalize_primnode((Node *) ((IndexScan *) plan)->indexqual,
&context);
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 935bc2b..276ad96 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -732,6 +732,32 @@ create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer
}
/*
+ * create_funnel_path
+ *
+ * Creates a path corresponding to a funnel scan, returning the
+ * pathnode.
+ */
+FunnelPath *
+create_funnel_path(PlannerInfo *root, RelOptInfo *rel, Path* subpath,
+ Relids required_outer, int nworkers)
+{
+ FunnelPath *pathnode = makeNode(FunnelPath);
+
+ pathnode->path.pathtype = T_Funnel;
+ pathnode->path.parent = rel;
+ pathnode->path.param_info = get_baserel_parampathinfo(root, rel,
+ required_outer);
+ pathnode->path.pathkeys = NIL; /* Funnel has unordered result */
+
+ pathnode->subpath = subpath;
+ pathnode->num_workers = nworkers;
+
+ cost_funnel(pathnode, root, rel, pathnode->path.param_info);
+
+ return pathnode;
+}
+
+/*
* create_index_path
* Creates a path node for an index scan.
*
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 000524d..4eb879b 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -103,6 +103,7 @@
#include "miscadmin.h"
#include "pg_getopt.h"
#include "pgstat.h"
+#include "optimizer/cost.h"
#include "postmaster/autovacuum.h"
#include "postmaster/bgworker_internals.h"
#include "postmaster/fork_process.h"
@@ -857,6 +858,12 @@ PostmasterMain(int argc, char *argv[])
ereport(ERROR,
(errmsg("WAL streaming (max_wal_senders > 0) requires wal_level \"archive\", \"hot_standby\", or \"logical\"")));
+ if (parallel_seqscan_degree >= MaxConnections)
+ {
+ write_stderr("%s: parallel_scan_degree must be less than max_connections\n", progname);
+ ExitPostmaster(1);
+ }
+
/*
* Other one-time internal sanity checks can go here, if they are fast.
* (Put any slow processing further down, after postmaster.pid creation.)
diff --git a/src/backend/storage/ipc/shm_mq.c b/src/backend/storage/ipc/shm_mq.c
index 0e60dbc..c78f165 100644
--- a/src/backend/storage/ipc/shm_mq.c
+++ b/src/backend/storage/ipc/shm_mq.c
@@ -746,6 +746,15 @@ shm_mq_detach(shm_mq *mq)
}
/*
+ * Get the shm_mq from handle.
+ */
+shm_mq *
+shm_mq_get_queue(shm_mq_handle *mqh)
+{
+ return mqh->mqh_queue;
+}
+
+/*
* Write bytes into a shared message queue.
*/
static shm_mq_result
diff --git a/src/backend/tcop/dest.c b/src/backend/tcop/dest.c
index bcf3895..57014ee 100644
--- a/src/backend/tcop/dest.c
+++ b/src/backend/tcop/dest.c
@@ -34,6 +34,7 @@
#include "commands/createas.h"
#include "commands/matview.h"
#include "executor/functions.h"
+#include "executor/tqueue.h"
#include "executor/tstoreReceiver.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
@@ -44,9 +45,10 @@
* dummy DestReceiver functions
* ----------------
*/
-static void
+static bool
donothingReceive(TupleTableSlot *slot, DestReceiver *self)
{
+ return true;
}
static void
@@ -129,6 +131,9 @@ CreateDestReceiver(CommandDest dest)
case DestTransientRel:
return CreateTransientRelDestReceiver(InvalidOid);
+
+ case DestTupleQueue:
+ return CreateTupleQueueDestReceiver();
}
/* should never get here */
@@ -162,6 +167,7 @@ EndCommand(const char *commandTag, CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -204,6 +210,7 @@ NullCommand(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
@@ -248,6 +255,7 @@ ReadyForQuery(CommandDest dest)
case DestCopyOut:
case DestSQLFunction:
case DestTransientRel:
+ case DestTupleQueue:
break;
}
}
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 7598318..f1542a0 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -42,6 +42,8 @@
#include "catalog/pg_type.h"
#include "commands/async.h"
#include "commands/prepare.h"
+#include "executor/execParallel.h"
+#include "executor/tqueue.h"
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqsignal.h"
@@ -1192,6 +1194,94 @@ exec_simple_query(const char *query_string)
}
/*
+ * exec_parallel_stmt
+ *
+ * Execute the plan for backend worker.
+ */
+void
+exec_parallel_stmt(ParallelStmt *parallelstmt)
+{
+ DestReceiver *receiver;
+ QueryDesc *queryDesc;
+ MemoryContext oldcontext;
+ MemoryContext plancontext;
+ BufferUsage bufusage_start;
+ BufferUsage bufusage_end = {0};
+
+ set_ps_display("SELECT", false);
+
+ /*
+ * Unlike exec_simple_query(), in backend worker we won't allow
+ * transaction control statements, so we can allow plancontext
+ * to be created in TopTransaction context.
+ */
+ plancontext = AllocSetContextCreate(CurrentMemoryContext,
+ "worker plan",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ oldcontext = MemoryContextSwitchTo(plancontext);
+
+ receiver = CreateDestReceiver(DestTupleQueue);
+ SetTupleQueueDestReceiverParams(receiver, parallelstmt->responseq);
+
+ /* Create a QueryDesc for the query */
+ queryDesc = CreateQueryDesc(parallelstmt->plannedstmt, "",
+ GetActiveSnapshot(), InvalidSnapshot,
+ receiver, parallelstmt->params,
+ parallelstmt->inst_options);
+
+ PushActiveSnapshot(queryDesc->snapshot);
+
+ /* call ExecutorStart to prepare the plan for execution */
+ ExecutorStart(queryDesc, 0);
+
+ PopulateParamExecParams(queryDesc, parallelstmt->serialized_param_exec_vals);
+
+ bufusage_start = pgBufferUsage;
+
+ /* run the plan */
+ ExecutorRun(queryDesc, ForwardScanDirection, 0L);
+
+ /*
+ * Calculate the buffer usage for this statement run, it is required
+ * by plugins like pg_stat_statements to report the total usage for
+ * statement execution.
+ */
+ BufferUsageAccumDiff(&bufusage_end,
+ &pgBufferUsage, &bufusage_start);
+
+ /* run cleanup too */
+ ExecutorFinish(queryDesc);
+
+ /* copy buffer usage into shared memory. */
+ memcpy(parallelstmt->buffer_usage,
+ &bufusage_end,
+ sizeof(BufferUsage));
+
+ /*
+ * copy intrumentation information into shared memory if requested
+ * by master backend.
+ */
+ if (parallelstmt->inst_options)
+ memcpy(parallelstmt->instrument,
+ queryDesc->planstate->instrument,
+ sizeof(Instrumentation));
+
+ ExecutorEnd(queryDesc);
+
+ PopActiveSnapshot();
+
+ FreeQueryDesc(queryDesc);
+
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);
+
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
* exec_parse_message
*
* Execute a "Parse" protocol message.
diff --git a/src/backend/tcop/pquery.c b/src/backend/tcop/pquery.c
index 9c14e8a..f2fb638 100644
--- a/src/backend/tcop/pquery.c
+++ b/src/backend/tcop/pquery.c
@@ -1121,7 +1121,13 @@ RunFromStore(Portal portal, ScanDirection direction, long count,
if (!ok)
break;
- (*dest->receiveSlot) (slot, dest);
+ /*
+ * 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.
+ */
+ if (!((*dest->receiveSlot) (slot, dest)))
+ break;
ExecClearTuple(slot);
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b3dac51..e4751f0 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -607,6 +607,8 @@ const char *const config_group_names[] =
gettext_noop("Statistics / Query and Index Statistics Collector"),
/* AUTOVACUUM */
gettext_noop("Autovacuum"),
+ /* PARALLEL_QUERY */
+ gettext_noop("parallel_seqscan_degree"),
/* CLIENT_CONN */
gettext_noop("Client Connection Defaults"),
/* CLIENT_CONN_STATEMENT */
@@ -2545,6 +2547,16 @@ static struct config_int ConfigureNamesInt[] =
},
{
+ {"parallel_seqscan_degree", PGC_SUSET, PARALLEL_QUERY,
+ gettext_noop("Sets the maximum number of simultaneously running backend worker processes."),
+ NULL
+ },
+ ¶llel_seqscan_degree,
+ 0, 0, MAX_BACKENDS,
+ NULL, NULL, NULL
+ },
+
+ {
{"autovacuum_work_mem", PGC_SIGHUP, RESOURCES_MEM,
gettext_noop("Sets the maximum memory to be used by each autovacuum worker process."),
NULL,
@@ -2721,6 +2733,26 @@ static struct config_real ConfigureNamesReal[] =
DEFAULT_CPU_OPERATOR_COST, 0, DBL_MAX,
NULL, NULL, NULL
},
+ {
+ {"cpu_tuple_comm_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "passing each tuple (row) from worker to master backend."),
+ NULL
+ },
+ &cpu_tuple_comm_cost,
+ DEFAULT_CPU_TUPLE_COMM_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
+ {
+ {"parallel_setup_cost", PGC_USERSET, QUERY_TUNING_COST,
+ gettext_noop("Sets the planner's estimate of the cost of "
+ "setting up environment (shared memory) for parallelism."),
+ NULL
+ },
+ ¶llel_setup_cost,
+ DEFAULT_PARALLEL_SETUP_COST, 0, DBL_MAX,
+ NULL, NULL, NULL
+ },
{
{"cursor_tuple_fraction", PGC_USERSET, QUERY_TUNING_OTHER,
diff --git a/src/backend/utils/misc/postgresql.conf.sample b/src/backend/utils/misc/postgresql.conf.sample
index e5d275d..9f75a5b 100644
--- a/src/backend/utils/misc/postgresql.conf.sample
+++ b/src/backend/utils/misc/postgresql.conf.sample
@@ -290,6 +290,8 @@
#cpu_tuple_cost = 0.01 # same scale as above
#cpu_index_tuple_cost = 0.005 # same scale as above
#cpu_operator_cost = 0.0025 # same scale as above
+#cpu_tuple_comm_cost = 0.1 # same scale as above
+#parallel_setup_cost = 0.0 # same scale as above
#effective_cache_size = 4GB
# - Genetic Query Optimizer -
@@ -500,6 +502,11 @@
# autovacuum, -1 means use
# vacuum_cost_limit
+#------------------------------------------------------------------------------
+# PARALLEL_QUERY PARAMETERS
+#------------------------------------------------------------------------------
+
+#parallel_seqscan_degree = 0 # max number of worker backend subprocesses
#------------------------------------------------------------------------------
# CLIENT CONNECTION DEFAULTS
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index 75e6b72..b3e3202 100644
--- 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,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
Relation stats_relation);
diff --git a/src/include/access/printtup.h b/src/include/access/printtup.h
index 46c4148..92ec882 100644
--- a/src/include/access/printtup.h
+++ b/src/include/access/printtup.h
@@ -25,11 +25,11 @@ extern void SendRowDescriptionMessage(TupleDesc typeinfo, List *targetlist,
extern void debugStartup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void debugtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool debugtup(TupleTableSlot *slot, DestReceiver *self);
/* XXX these are really in executor/spi.c */
extern void spi_dest_startup(DestReceiver *self, int operation,
TupleDesc typeinfo);
-extern void spi_printtup(TupleTableSlot *slot, DestReceiver *self);
+extern bool spi_printtup(TupleTableSlot *slot, DestReceiver *self);
#endif /* PRINTTUP_H */
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
new file mode 100644
index 0000000..b6a09dd
--- /dev/null
+++ b/src/include/executor/execParallel.h
@@ -0,0 +1,65 @@
+/*--------------------------------------------------------------------
+ * execParallel.h
+ * POSTGRES parallel execution interface
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/include/executor/execParallel.h
+ *--------------------------------------------------------------------
+ */
+#ifndef EXECPARALLEL_H
+#define EXECPARALLEL_H
+
+/*---------------------------------------------------------------------
+ * External module API.
+ *---------------------------------------------------------------------
+ */
+
+#include "libpq/pqmq.h"
+#include "nodes/execnodes.h"
+#include "nodes/parsenodes.h"
+#include "nodes/plannodes.h"
+
+/* Table-of-contents constants for our dynamic shared memory segment. */
+#define PARALLEL_KEY_PLANNEDSTMT 0
+#define PARALLEL_KEY_PARAMS 1
+#define PARALLEL_KEY_PARAMS_EXEC 2
+#define PARALLEL_KEY_BUFF_USAGE 3
+#define PARALLEL_KEY_INST_OPTIONS 4
+#define PARALLEL_KEY_INST_INFO 5
+#define PARALLEL_KEY_TUPLE_QUEUE 6
+#define PARALLEL_KEY_SCAN 7
+
+extern int parallel_seqscan_degree;
+
+/* worker statement required for parallel execution. */
+typedef struct ParallelStmt
+{
+ PlannedStmt *plannedstmt;
+ ParamListInfo params;
+ List *serialized_param_exec_vals;
+ shm_mq_handle *responseq;
+ int inst_options;
+ char *instrument;
+ char *buffer_usage;
+} ParallelStmt;
+
+extern void InitializeParallelWorkers(PlanState *planstate,
+ List *serialized_param_exec_vals,
+ EState *estate,
+ char **inst_options_space,
+ char **buffer_usage_space,
+ shm_mq_handle ***responseqp,
+ ParallelContext **pcxtp,
+ int nWorkers);
+extern shm_toc *GetParallelShmToc(void);
+extern bool ExecParallelEstimate(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
+ Size *pscan_size);
+extern bool ExecParallelBufferUsageAccum(Node *node);
+extern void ExecAssociateBufferStatsToDSM(BufferUsage *buf_usage,
+ ParallelStmt *parallel_stmt);
+#endif /* EXECPARALLEL_H */
diff --git a/src/include/executor/executor.h b/src/include/executor/executor.h
index 193a654..963e656 100644
--- a/src/include/executor/executor.h
+++ b/src/include/executor/executor.h
@@ -273,6 +273,8 @@ extern TupleDesc ExecCleanTypeFromTL(List *targetList, bool hasoid);
extern TupleDesc ExecTypeFromExprList(List *exprList);
extern void ExecTypeSetColNames(TupleDesc typeInfo, List *namesList);
extern void UpdateChangedParamSet(PlanState *node, Bitmapset *newchg);
+extern void PopulateParamExecParams(QueryDesc *queryDesc,
+ List *serialized_param_exec_vals);
typedef struct TupOutputState
{
diff --git a/src/include/executor/instrument.h b/src/include/executor/instrument.h
index c9a2129..0c7847d 100644
--- a/src/include/executor/instrument.h
+++ b/src/include/executor/instrument.h
@@ -69,5 +69,12 @@ extern Instrumentation *InstrAlloc(int n, int instrument_options);
extern void InstrStartNode(Instrumentation *instr);
extern void InstrStopNode(Instrumentation *instr, double nTuples);
extern void InstrEndLoop(Instrumentation *instr);
+extern void InstrAggNode(Instrumentation *instr1, Instrumentation *instr2);
+extern void
+ InstrAggBufferUsage(BufferUsage *buffer_usage_dst, BufferUsage *buffer_usage_add);
+extern void BufferUsageAccumDiff(BufferUsage *dst,
+ const BufferUsage *add,
+ const BufferUsage *sub);
+extern void BufferUsageAdd(BufferUsage *dst, const BufferUsage *add);
#endif /* INSTRUMENT_H */
diff --git a/src/include/executor/nodeFunnel.h b/src/include/executor/nodeFunnel.h
new file mode 100644
index 0000000..b996244
--- /dev/null
+++ b/src/include/executor/nodeFunnel.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodeFunnel.h
+ * prototypes for nodeFunnel.c
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodeFunnel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEFUNNEL_H
+#define NODEFUNNEL_H
+
+#include "nodes/execnodes.h"
+
+extern FunnelState *ExecInitFunnel(Funnel *node, EState *estate, int eflags);
+extern TupleTableSlot *ExecFunnel(FunnelState *node);
+extern void ExecEndFunnel(FunnelState *node);
+extern void FinishParallelSetupAndAccumStats(FunnelState *node);
+extern void ExecReScanFunnel(FunnelState *node);
+
+#endif /* NODEFUNNEL_H */
diff --git a/src/include/executor/nodeSubplan.h b/src/include/executor/nodeSubplan.h
index 3732ad4..21c745e 100644
--- a/src/include/executor/nodeSubplan.h
+++ b/src/include/executor/nodeSubplan.h
@@ -24,4 +24,7 @@ extern void ExecReScanSetParamPlan(SubPlanState *node, PlanState *parent);
extern void ExecSetParamPlan(SubPlanState *node, ExprContext *econtext);
+extern List *
+ExecAndFormSerializeParamExec(ExprContext *econtext, Bitmapset *params);
+
#endif /* NODESUBPLAN_H */
diff --git a/src/include/executor/tqueue.h b/src/include/executor/tqueue.h
new file mode 100644
index 0000000..ce16936
--- /dev/null
+++ b/src/include/executor/tqueue.h
@@ -0,0 +1,35 @@
+/*-------------------------------------------------------------------------
+ *
+ * tqueue.h
+ * prototypes for tqueue.c
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/tqueue.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef TQUEUE_H
+#define TQUEUE_H
+
+#include "storage/shm_mq.h"
+#include "tcop/dest.h"
+
+/* Use this to send tuples to a shm_mq. */
+extern DestReceiver *CreateTupleQueueDestReceiver(void);
+extern void SetTupleQueueDestReceiverParams(DestReceiver *self,
+ shm_mq_handle *handle);
+
+/* Use these to receive tuples from a shm_mq. */
+typedef struct TupleQueueFunnel TupleQueueFunnel;
+extern TupleQueueFunnel *CreateTupleQueueFunnel(void);
+extern void TupleQueueFunnelShutdown(TupleQueueFunnel *funnel);
+extern void DestroyTupleQueueFunnel(TupleQueueFunnel *funnel);
+extern void RegisterTupleQueueOnFunnel(TupleQueueFunnel *, shm_mq_handle *);
+extern HeapTuple TupleQueueFunnelNext(TupleQueueFunnel *, bool nowait,
+ bool *done);
+
+#endif /* TQUEUE_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 5796de8..8f10c4e 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -16,7 +16,9 @@
#include "access/genam.h"
#include "access/heapam.h"
+#include "access/parallel.h"
#include "executor/instrument.h"
+#include "executor/tqueue.h"
#include "lib/pairingheap.h"
#include "nodes/params.h"
#include "nodes/plannodes.h"
@@ -401,6 +403,18 @@ typedef struct EState
List *es_auxmodifytables; /* List of secondary ModifyTableStates */
/*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
+
+ /*
+ * This is required to collect buffer usage stats from parallel
+ * workers when requested by plugins.
+ */
+ bool total_time; /* total time spent in ExecutorRun */
+
+ /*
* this ExprContext is for per-output-tuple operations, such as constraint
* checks and index-value computations. It will be reset for each output
* tuple. Note that it will be created only if needed.
@@ -1047,6 +1061,11 @@ typedef struct PlanState
* State for management of parameter-change-driven rescanning
*/
Bitmapset *chgParam; /* set of IDs of changed Params */
+ /*
+ * This is required for parallel plan execution to fetch the
+ * information from dsm.
+ */
+ shm_toc *toc;
/*
* Other run-time state needed by most if not all node types.
@@ -1273,6 +1292,35 @@ typedef struct SampleScanState
} SampleScanState;
/*
+ * FunnelState extends ScanState by storing additional information
+ * related to parallel workers.
+ * pcxt parallel context for managing generic state information
+ * required for parallelism.
+ * responseq shared memory queues to receive data from workers.
+ * funnel maintains the runtime information about queue's used to
+ * receive data from parallel workers.
+ * inst_options_space to accumulate instrumentation information from all
+ * parallel workers.
+ * buffer_usage_space to accumulate buffer usage information from all
+ * parallel workers.
+ * fs_workersReady indicates that workers are launched.
+ * all_workers_done indicates that all the data from workers has been received.
+ * local_scan_done indicates that local scan is compleleted.
+ */
+typedef struct FunnelState
+{
+ ScanState ss; /* its first field is NodeTag */
+ ParallelContext *pcxt;
+ shm_mq_handle **responseq;
+ TupleQueueFunnel *funnel;
+ char *inst_options_space;
+ char *buffer_usage_space;
+ bool fs_workersReady;
+ bool all_workers_done;
+ bool local_scan_done;
+} FunnelState;
+
+/*
* These structs store information about index quals that don't have simple
* constant right-hand sides. See comments for ExecIndexBuildScanKeys()
* for discussion.
diff --git a/src/include/nodes/nodeFuncs.h b/src/include/nodes/nodeFuncs.h
index 7b1b1d6..df00d3d 100644
--- a/src/include/nodes/nodeFuncs.h
+++ b/src/include/nodes/nodeFuncs.h
@@ -13,6 +13,7 @@
#ifndef NODEFUNCS_H
#define NODEFUNCS_H
+#include "access/parallel.h"
#include "nodes/parsenodes.h"
@@ -63,4 +64,7 @@ extern Node *query_or_expression_tree_mutator(Node *node, Node *(*mutator) (),
extern bool raw_expression_tree_walker(Node *node, bool (*walker) (),
void *context);
+extern bool planstate_tree_walker(Node *node, ParallelContext *pcxt,
+ bool (*walker) (), void *context);
+
#endif /* NODEFUNCS_H */
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index 748e434..f456004 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -52,6 +52,7 @@ typedef enum NodeTag
T_Scan,
T_SeqScan,
T_SampleScan,
+ T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
T_BitmapIndexScan,
@@ -99,6 +100,7 @@ typedef enum NodeTag
T_ScanState,
T_SeqScanState,
T_SampleScanState,
+ T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
T_BitmapIndexScanState,
@@ -223,6 +225,7 @@ typedef enum NodeTag
T_IndexOptInfo,
T_ParamPathInfo,
T_Path,
+ T_FunnelPath,
T_IndexPath,
T_BitmapHeapPath,
T_BitmapAndPath,
diff --git a/src/include/nodes/params.h b/src/include/nodes/params.h
index a0f7dd0..21c6f7a 100644
--- a/src/include/nodes/params.h
+++ b/src/include/nodes/params.h
@@ -14,6 +14,8 @@
#ifndef PARAMS_H
#define PARAMS_H
+#include "nodes/pg_list.h"
+
/* To avoid including a pile of parser headers, reference ParseState thus: */
struct ParseState;
@@ -96,11 +98,47 @@ typedef struct ParamExecData
{
void *execPlan; /* should be "SubPlanState *" */
Datum value;
+ /*
+ * parameter's datatype, or 0. This is required so that
+ * datum value can be read and used for other purposes like
+ * passing it to worker backend via shared memory. This is
+ * required only for evaluation of initPlan's, however for
+ * consistency we set this for Subplan as well. We left it
+ * for other cases like CTE or RecursiveUnion cases where this
+ * structure is not used for evaluation of subplans.
+ */
+ Oid ptype;
bool isnull;
} ParamExecData;
+/*
+ * This structure is used to pass PARAM_EXEC parameters to backend
+ * workers. For each PARAM_EXEC parameter, pass this structure
+ * followed by value except for pass-by-value parameters.
+ */
+typedef struct SerializedParamExecData
+{
+ int paramid; /* parameter id of this param */
+ Size length; /* length of parameter value */
+ Oid ptype; /* parameter's datatype, or 0 */
+ Datum value;
+ bool isnull;
+} SerializedParamExecData;
+
/* Functions found in src/backend/nodes/params.c */
extern ParamListInfo copyParamList(ParamListInfo from);
+extern Size
+EstimateBoundParametersSpace(ParamListInfo params);
+extern void
+SerializeBoundParams(ParamListInfo params, Size maxsize, char *start_address);
+extern ParamListInfo RestoreBoundParams(char *start_address);
+extern Size
+EstimateExecParametersSpace(List *serialized_param_exec_vals);
+extern void
+SerializeExecParams(List *serialized_param_exec_vals, Size maxsize,
+ char *start_address);
+List *
+RestoreExecParams(char *start_address);
#endif /* PARAMS_H */
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index cc259f1..69302af 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -296,6 +296,16 @@ typedef struct SampleScan
struct TableSampleClause *tablesample;
} SampleScan;
+/* ------------
+ * Funnel node
+ * ------------
+ */
+typedef struct Funnel
+{
+ Scan scan;
+ int num_workers;
+} Funnel;
+
/* ----------------
* index scan node
*
diff --git a/src/include/nodes/relation.h b/src/include/nodes/relation.h
index 79bed33..f2faa1f 100644
--- a/src/include/nodes/relation.h
+++ b/src/include/nodes/relation.h
@@ -761,6 +761,13 @@ typedef struct Path
/* pathkeys is a List of PathKey nodes; see above */
} Path;
+typedef struct FunnelPath
+{
+ Path path;
+ Path *subpath; /* path for each worker */
+ int num_workers;
+} FunnelPath;
+
/* Macro for extracting a path's parameterization relids; beware double eval */
#define PATH_REQ_OUTER(path) \
((path)->param_info ? (path)->param_info->ppi_req_outer : (Relids) NULL)
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index dd43e45..994ea83 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -26,6 +26,13 @@
#define DEFAULT_CPU_TUPLE_COST 0.01
#define DEFAULT_CPU_INDEX_TUPLE_COST 0.005
#define DEFAULT_CPU_OPERATOR_COST 0.0025
+#define DEFAULT_CPU_TUPLE_COMM_COST 0.1
+/*
+ * XXX - We need some experiments to know what could be
+ * appropriate default values for parallel setup and startup
+ * cost.
+ */
+#define DEFAULT_PARALLEL_SETUP_COST 0.0
#define DEFAULT_EFFECTIVE_CACHE_SIZE 524288 /* measured in pages */
@@ -48,8 +55,11 @@ extern PGDLLIMPORT double random_page_cost;
extern PGDLLIMPORT double cpu_tuple_cost;
extern PGDLLIMPORT double cpu_index_tuple_cost;
extern PGDLLIMPORT double cpu_operator_cost;
+extern PGDLLIMPORT double cpu_tuple_comm_cost;
+extern PGDLLIMPORT double parallel_setup_cost;
extern PGDLLIMPORT int effective_cache_size;
extern Cost disable_cost;
+extern int parallel_seqscan_degree;
extern bool enable_seqscan;
extern bool enable_indexscan;
extern bool enable_indexonlyscan;
@@ -70,6 +80,8 @@ extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info);
extern void cost_index(IndexPath *path, PlannerInfo *root,
double loop_count);
extern void cost_bitmap_heap_scan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 161644c..9d31b93 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -34,6 +34,9 @@ extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
extern Path *create_samplescan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern FunnelPath *create_funnel_path(PlannerInfo *root,
+ RelOptInfo *rel, Path *subpath, Relids required_outer,
+ int nworkers);
extern IndexPath *create_index_path(PlannerInfo *root,
IndexOptInfo *index,
List *indexclauses,
diff --git a/src/include/optimizer/planmain.h b/src/include/optimizer/planmain.h
index 52b077a..67a8582 100644
--- a/src/include/optimizer/planmain.h
+++ b/src/include/optimizer/planmain.h
@@ -133,6 +133,7 @@ extern bool query_is_distinct_for(Query *query, List *colnos, List *opids);
*/
extern Plan *set_plan_references(PlannerInfo *root, Plan *plan);
extern void fix_opfuncids(Node *node);
+extern void fix_node_funcids(Plan *node);
extern void set_opfuncid(OpExpr *opexpr);
extern void set_sa_opfuncid(ScalarArrayOpExpr *opexpr);
extern void record_plan_function_dependency(PlannerInfo *root, Oid funcid);
diff --git a/src/include/optimizer/planner.h b/src/include/optimizer/planner.h
index b10a504..dea968a 100644
--- a/src/include/optimizer/planner.h
+++ b/src/include/optimizer/planner.h
@@ -14,6 +14,7 @@
#ifndef PLANNER_H
#define PLANNER_H
+#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
#include "nodes/relation.h"
@@ -29,6 +30,8 @@ extern PlannedStmt *planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
extern PlannedStmt *standard_planner(Query *parse, int cursorOptions,
ParamListInfo boundParams);
+extern PlannedStmt *create_parallel_worker_plannedstmt(Plan *plan,
+ List *rangetable, int num_exec_params);
extern Plan *subquery_planner(PlannerGlobal *glob, Query *parse,
PlannerInfo *parent_root,
diff --git a/src/include/storage/shm_mq.h b/src/include/storage/shm_mq.h
index 1a2ba04..7621a35 100644
--- a/src/include/storage/shm_mq.h
+++ b/src/include/storage/shm_mq.h
@@ -65,6 +65,9 @@ extern void shm_mq_set_handle(shm_mq_handle *, BackgroundWorkerHandle *);
/* Break connection. */
extern void shm_mq_detach(shm_mq *);
+/* Get the shm_mq from handle. */
+extern shm_mq *shm_mq_get_queue(shm_mq_handle *mqh);
+
/* Send or receive messages. */
extern shm_mq_result shm_mq_send(shm_mq_handle *mqh,
Size nbytes, const void *data, bool nowait);
diff --git a/src/include/tcop/dest.h b/src/include/tcop/dest.h
index 5bcca3f..91acd60 100644
--- a/src/include/tcop/dest.h
+++ b/src/include/tcop/dest.h
@@ -94,7 +94,8 @@ typedef enum
DestIntoRel, /* results sent to relation (SELECT INTO) */
DestCopyOut, /* results sent to COPY TO code */
DestSQLFunction, /* results sent to SQL-language func mgr */
- DestTransientRel /* results sent to transient relation */
+ DestTransientRel, /* results sent to transient relation */
+ DestTupleQueue /* results sent to tuple queue */
} CommandDest;
/* ----------------
@@ -103,7 +104,9 @@ typedef enum
* pointers that the executor must call.
*
* Note: the receiveSlot routine must be passed a slot containing a TupleDesc
- * identical to the one given to the rStartup routine.
+ * identical to the one given to the rStartup routine. It returns bool where
+ * a "true" value means "continue processing" and a "false" value means
+ * "stop early, just as if we'd reached the end of the scan".
* ----------------
*/
typedef struct _DestReceiver DestReceiver;
@@ -111,7 +114,7 @@ typedef struct _DestReceiver DestReceiver;
struct _DestReceiver
{
/* Called for each tuple to be output: */
- void (*receiveSlot) (TupleTableSlot *slot,
+ bool (*receiveSlot) (TupleTableSlot *slot,
DestReceiver *self);
/* Per-executor-run initialization and shutdown: */
void (*rStartup) (DestReceiver *self,
diff --git a/src/include/tcop/tcopprot.h b/src/include/tcop/tcopprot.h
index 96c5b8b..6f319c1 100644
--- a/src/include/tcop/tcopprot.h
+++ b/src/include/tcop/tcopprot.h
@@ -19,6 +19,7 @@
#ifndef TCOPPROT_H
#define TCOPPROT_H
+#include "executor/execParallel.h"
#include "nodes/params.h"
#include "nodes/parsenodes.h"
#include "nodes/plannodes.h"
@@ -84,5 +85,6 @@ extern void set_debug_options(int debug_flag,
extern bool set_plan_disabling_options(const char *arg,
GucContext context, GucSource source);
extern const char *get_stats_option_name(const char *arg);
+extern void exec_parallel_stmt(ParallelStmt *parallelscan);
#endif /* TCOPPROT_H */
diff --git a/src/include/utils/guc_tables.h b/src/include/utils/guc_tables.h
index 7a58ddb..3505d31 100644
--- a/src/include/utils/guc_tables.h
+++ b/src/include/utils/guc_tables.h
@@ -85,6 +85,7 @@ enum config_group
STATS_MONITORING,
STATS_COLLECTOR,
AUTOVACUUM,
+ PARALLEL_QUERY,
CLIENT_CONN,
CLIENT_CONN_STATEMENT,
CLIENT_CONN_LOCALE,
parallel_seqscan_partialseqscan_v17.patchapplication/octet-stream; name=parallel_seqscan_partialseqscan_v17.patchDownload
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 3701d8e..831329a 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -63,6 +63,7 @@
#include "storage/predicate.h"
#include "storage/procarray.h"
#include "storage/smgr.h"
+#include "storage/spin.h"
#include "storage/standby.h"
#include "utils/datum.h"
#include "utils/inval.h"
@@ -80,12 +81,16 @@ bool synchronize_seqscans = true;
static HeapScanDesc heap_beginscan_internal(Relation relation,
Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat,
bool allow_sync,
bool allow_pagemode,
bool is_bitmapscan,
bool is_samplescan,
bool temp_snap);
+static BlockNumber heap_parallelscan_nextpage(HeapScanDesc scan,
+ bool *pscan_finished);
+static void heap_parallelscan_initialize_startblock(HeapScanDesc scan);
static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup,
TransactionId xid, CommandId cid, int options);
static XLogRecPtr log_heap_update(Relation reln, Buffer oldbuf,
@@ -226,7 +231,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool keep_startblock)
* results for a non-MVCC snapshot, the caller must hold some higher-level
* lock that ensures the interesting tuple(s) won't change.)
*/
- scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
+ if (scan->rs_parallel != NULL)
+ scan->rs_nblocks = scan->rs_parallel->phs_nblocks;
+ else
+ scan->rs_nblocks = RelationGetNumberOfBlocks(scan->rs_rd);
/*
* If the table is large relative to NBuffers, use a bulk-read access
@@ -272,7 +280,10 @@ initscan(HeapScanDesc scan, ScanKey key, bool keep_startblock)
else if (allow_sync && synchronize_seqscans)
{
scan->rs_syncscan = true;
- scan->rs_startblock = ss_get_location(scan->rs_rd, scan->rs_nblocks);
+ if (scan->rs_parallel != NULL)
+ heap_parallelscan_initialize_startblock(scan);
+ else
+ scan->rs_startblock = ss_get_location(scan->rs_rd, scan->rs_nblocks);
}
else
{
@@ -496,7 +507,32 @@ heapgettup(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ bool pscan_finished;
+
+ page = heap_parallelscan_nextpage(scan, &pscan_finished);
+
+ /*
+ * Return NULL if the scan is finished. It can so happen that
+ * by the time one of workers started the scan, others have
+ * already completed scanning the relation, so this worker won't
+ * need to perform scan. Report scan location before finishing the
+ * scan so that the final state of the position hint is back at the
+ * start of the rel.
+ */
+ if (pscan_finished)
+ {
+ if (scan->rs_syncscan)
+ ss_report_location(scan->rs_rd, page);
+
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineoff = FirstOffsetNumber; /* first offnum */
scan->rs_inited = true;
@@ -519,6 +555,9 @@ heapgettup(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -671,11 +710,22 @@ heapgettup(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks == 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ bool pscan_finished = false;
+
+ page = heap_parallelscan_nextpage(scan, &pscan_finished);
+ finished = pscan_finished;
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -773,7 +823,32 @@ heapgettup_pagemode(HeapScanDesc scan,
tuple->t_data = NULL;
return;
}
- page = scan->rs_startblock; /* first page */
+ if (scan->rs_parallel != NULL)
+ {
+ bool pscan_finished;
+
+ page = heap_parallelscan_nextpage(scan, &pscan_finished);
+
+ /*
+ * Return NULL if the scan is finished. It can so happen that
+ * by the time one of workers started the scan, others have
+ * already completed scanning the relation, so this worker won't
+ * need to perform scan. Report scan location before finishing the
+ * scan so that the final state of the position hint is back at the
+ * start of the rel.
+ */
+ if (pscan_finished)
+ {
+ if (scan->rs_syncscan)
+ ss_report_location(scan->rs_rd, page);
+
+ Assert(!BufferIsValid(scan->rs_cbuf));
+ tuple->t_data = NULL;
+ return;
+ }
+ }
+ else
+ page = scan->rs_startblock; /* first page */
heapgetpage(scan, page);
lineindex = 0;
scan->rs_inited = true;
@@ -793,6 +868,9 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else if (backward)
{
+ /* backward parallel scan not supported */
+ Assert(scan->rs_parallel == NULL);
+
if (!scan->rs_inited)
{
/*
@@ -934,11 +1012,22 @@ heapgettup_pagemode(HeapScanDesc scan,
}
else
{
- page++;
- if (page >= scan->rs_nblocks)
- page = 0;
- finished = (page == scan->rs_startblock) ||
- (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks == 0 : false);
+ if (scan->rs_parallel != NULL)
+ {
+ bool pscan_finished = false;
+
+ page = heap_parallelscan_nextpage(scan, &pscan_finished);
+ finished = pscan_finished;
+ }
+ else
+ {
+ page++;
+ if (page >= scan->rs_nblocks)
+ page = 0;
+
+ finished = (page == scan->rs_startblock) ||
+ (scan->rs_numblocks != InvalidBlockNumber ? --scan->rs_numblocks <= 0 : false);
+ }
/*
* Report our new scan position for synchronization purposes. We
@@ -1341,7 +1430,7 @@ HeapScanDesc
heap_beginscan(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, false);
}
@@ -1351,7 +1440,7 @@ heap_beginscan_catalog(Relation relation, int nkeys, ScanKey key)
Oid relid = RelationGetRelid(relation);
Snapshot snapshot = RegisterSnapshot(GetCatalogSnapshot(relid));
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
true, true, true, false, false, true);
}
@@ -1360,7 +1449,7 @@ heap_beginscan_strat(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, true,
false, false, false);
}
@@ -1369,7 +1458,7 @@ HeapScanDesc
heap_beginscan_bm(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
false, false, true, true, false, false);
}
@@ -1378,7 +1467,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
bool allow_strat, bool allow_sync, bool allow_pagemode)
{
- return heap_beginscan_internal(relation, snapshot, nkeys, key,
+ return heap_beginscan_internal(relation, snapshot, nkeys, key, NULL,
allow_strat, allow_sync, allow_pagemode,
false, true, false);
}
@@ -1386,6 +1475,7 @@ heap_beginscan_sampling(Relation relation, Snapshot snapshot,
static HeapScanDesc
heap_beginscan_internal(Relation relation, Snapshot snapshot,
int nkeys, ScanKey key,
+ ParallelHeapScanDesc parallel_scan,
bool allow_strat,
bool allow_sync,
bool allow_pagemode,
@@ -1418,6 +1508,7 @@ heap_beginscan_internal(Relation relation, Snapshot snapshot,
scan->rs_allow_strat = allow_strat;
scan->rs_allow_sync = allow_sync;
scan->rs_temp_snap = temp_snap;
+ scan->rs_parallel = parallel_scan;
/*
* we can use page-at-a-time mode if it's an MVCC-safe snapshot
@@ -1532,6 +1623,159 @@ heap_endscan(HeapScanDesc scan)
}
/* ----------------
+ * heap_parallelscan_estimate - estimate storage for ParallelHeapScanDesc
+ *
+ * Sadly, this doesn't reduce to a constant, because the size required
+ * to serialize the snapshot can vary.
+ * ----------------
+ */
+Size
+heap_parallelscan_estimate(Snapshot snapshot)
+{
+ return add_size(offsetof(ParallelHeapScanDescData, phs_snapshot_data),
+ EstimateSnapshotSpace(snapshot));
+}
+
+/* ----------------
+ * heap_parallelscan_initialize - initialize ParallelHeapScanDesc
+ *
+ * Must allow as many bytes of shared memory as returned by
+ * heap_parallelscan_estimate. Call this just once in the leader
+ * process; then, individual workers attach via heap_beginscan_parallel.
+ * ----------------
+ */
+void
+heap_parallelscan_initialize(ParallelHeapScanDesc target, Relation relation,
+ Snapshot snapshot)
+{
+ target->phs_relid = RelationGetRelid(relation);
+ target->phs_nblocks = RelationGetNumberOfBlocks(relation);
+ SpinLockInit(&target->phs_mutex);
+ target->phs_cblock = InvalidBlockNumber;
+ target->phs_firstpass = true;
+ SerializeSnapshot(snapshot, target->phs_snapshot_data);
+}
+
+/* ----------------
+ * heap_parallelscan_initialize_startblock - initialize the startblock for
+ * parallel scan.
+ *
+ * Only the first worker of parallel scan will initialize the start
+ * block for scan and others will use that information to indicate
+ * the end of scan.
+ * ----------------
+ */
+static void
+heap_parallelscan_initialize_startblock(HeapScanDesc scan)
+{
+ ParallelHeapScanDesc parallel_scan;
+
+ Assert(scan->rs_parallel);
+
+ parallel_scan = scan->rs_parallel;
+
+ /*
+ * InvalidBlockNumber indicates that this initialization is done for
+ * first worker.
+ */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock == InvalidBlockNumber)
+ {
+ scan->rs_startblock = ss_get_location(scan->rs_rd, scan->rs_nblocks);
+ parallel_scan->phs_cblock = scan->rs_startblock;
+ parallel_scan->phs_startblock = scan->rs_startblock;
+ }
+ else
+ scan->rs_startblock = parallel_scan->phs_startblock;
+ SpinLockRelease(¶llel_scan->phs_mutex);
+}
+
+/* ----------------
+ * heap_parallelscan_nextpage - get the next page to scan
+ *
+ * Scanning till the position from where the parallel scan has started
+ * indicates end of scan. Note, however, that other backends could still
+ * be scanning if they grabbed a page to scan and aren't done with it yet.
+ * Resets the current position for parallel scan to the begining of
+ * relation, if next page to scan is greater than total number of pages in
+ * relation.
+ * ----------------
+ */
+static BlockNumber
+heap_parallelscan_nextpage(HeapScanDesc scan,
+ bool *pscan_finished)
+{
+ BlockNumber page = InvalidBlockNumber;
+ ParallelHeapScanDesc parallel_scan;
+
+ Assert(scan->rs_parallel);
+
+ parallel_scan = scan->rs_parallel;
+
+ *pscan_finished = false;
+
+ /* we treat InvalidBlockNumber specially here to avoid overflow */
+ SpinLockAcquire(¶llel_scan->phs_mutex);
+ if (parallel_scan->phs_cblock != InvalidBlockNumber)
+ page = parallel_scan->phs_cblock++;
+
+ if (page >= scan->rs_nblocks)
+ {
+ parallel_scan->phs_cblock = 0;
+ page = parallel_scan->phs_cblock++;
+ }
+
+ /*
+ * scan position will be same as start position once during start
+ * of scan and then at end of scan.
+ */
+ if (parallel_scan->phs_firstpass && page == parallel_scan->phs_startblock)
+ parallel_scan->phs_firstpass = false;
+ else if (!parallel_scan->phs_firstpass && page == parallel_scan->phs_startblock)
+ {
+ *pscan_finished = true;
+ parallel_scan->phs_cblock--;
+ }
+ SpinLockRelease(¶llel_scan->phs_mutex);
+
+ return page;
+}
+
+/* ----------------
+ * heap_beginscan_parallel - join a parallel scan
+ *
+ * Caller must hold a suitable lock on the correct relation.
+ * ----------------
+ */
+HeapScanDesc
+heap_beginscan_parallel(Relation relation, ParallelHeapScanDesc parallel_scan)
+{
+ Snapshot snapshot;
+
+ Assert(RelationGetRelid(relation) == parallel_scan->phs_relid);
+ snapshot = RestoreSnapshot(parallel_scan->phs_snapshot_data);
+ RegisterSnapshot(snapshot);
+
+ return heap_beginscan_internal(relation, snapshot, 0, NULL, parallel_scan,
+ true, true, true, false, false, true);
+}
+
+/* ----------------
+ * heap_parallel_rescan - restart a parallel relation scan
+ * ----------------
+ */
+void
+heap_parallel_rescan(ParallelHeapScanDesc pscan,
+ HeapScanDesc scan)
+{
+ if (pscan != NULL)
+ scan->rs_parallel = pscan;
+
+ heap_rescan(scan, /* scan desc */
+ NULL); /* new scan keys */
+}
+
+/* ----------------
* heap_getnext - retrieve next tuple in scan
*
* Fix to work with index relations.
diff --git a/src/backend/commands/explain.c b/src/backend/commands/explain.c
index 69d3b34..1a0a550 100644
--- a/src/backend/commands/explain.c
+++ b/src/backend/commands/explain.c
@@ -734,6 +734,7 @@ ExplainPreScanNode(PlanState *planstate, Bitmapset **rels_used)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
@@ -942,6 +943,9 @@ ExplainNode(PlanState *planstate, List *ancestors,
case T_SampleScan:
pname = sname = "Sample Scan";
break;
+ case T_PartialSeqScan:
+ pname = sname = "Partial Seq Scan";
+ break;
case T_Funnel:
pname = sname = "Funnel";
break;
@@ -1095,6 +1099,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_Funnel:
case T_BitmapHeapScan:
case T_TidScan:
@@ -1370,6 +1375,7 @@ ExplainNode(PlanState *planstate, List *ancestors,
planstate, ancestors, es);
/* FALL THRU to print additional fields the same as SeqScan */
case T_SeqScan:
+ case T_PartialSeqScan:
case T_ValuesScan:
case T_CteScan:
case T_WorkTableScan:
@@ -2446,6 +2452,7 @@ ExplainTargetRel(Plan *plan, Index rti, ExplainState *es)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
diff --git a/src/backend/executor/Makefile b/src/backend/executor/Makefile
index 8037417..be1f47e 100644
--- a/src/backend/executor/Makefile
+++ b/src/backend/executor/Makefile
@@ -20,8 +20,8 @@ OBJS = execAmi.o execCurrent.o execGrouping.o execIndexing.o execJunk.o \
nodeHash.o nodeHashjoin.o nodeIndexscan.o nodeIndexonlyscan.o \
nodeLimit.o nodeLockRows.o \
nodeMaterial.o nodeMergeAppend.o nodeMergejoin.o nodeModifyTable.o \
- nodeNestloop.o nodeFunctionscan.o nodeRecursiveunion.o nodeResult.o \
- nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
+ nodeNestloop.o nodeFunctionscan.o nodePartialSeqscan.o nodeRecursiveunion.o \
+ nodeResult.o nodeSamplescan.o nodeSeqscan.o nodeSetOp.o nodeSort.o nodeUnique.o \
nodeValuesscan.o nodeCtescan.o nodeWorktablescan.o \
nodeGroup.o nodeSubplan.o nodeSubqueryscan.o nodeTidscan.o \
nodeForeignscan.o nodeWindowAgg.o tqueue.o tstoreReceiver.o spi.o
diff --git a/src/backend/executor/execAmi.c b/src/backend/executor/execAmi.c
index 4915151..dc45c20 100644
--- a/src/backend/executor/execAmi.c
+++ b/src/backend/executor/execAmi.c
@@ -38,6 +38,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
#include "executor/nodeSamplescan.h"
@@ -161,6 +162,10 @@ ExecReScan(PlanState *node)
ExecReScanSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecReScanPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
case T_FunnelState:
ExecReScanFunnel((FunnelState *) node);
break;
@@ -473,6 +478,7 @@ ExecSupportsBackwardScan(Plan *node)
return false;
case T_Funnel:
+ case T_PartialSeqScan:
return false;
case T_IndexScan:
diff --git a/src/backend/executor/execCurrent.c b/src/backend/executor/execCurrent.c
index 650fcc5..7a44462 100644
--- a/src/backend/executor/execCurrent.c
+++ b/src/backend/executor/execCurrent.c
@@ -262,6 +262,7 @@ search_plan_tree(PlanState *node, Oid table_oid)
*/
case T_SeqScanState:
case T_SampleScanState:
+ case T_PartialSeqScanState:
case T_FunnelState:
case T_IndexScanState:
case T_IndexOnlyScanState:
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 806f060..b1cbe7e 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -17,6 +17,7 @@
#include "executor/execParallel.h"
#include "executor/nodeFunnel.h"
+#include "executor/nodePartialSeqscan.h"
#include "nodes/nodeFuncs.h"
#include "optimizer/planmain.h"
#include "optimizer/planner.h"
@@ -295,6 +296,24 @@ ExecParallelEstimate(Node *node, ParallelContext *pcxt,
*/
switch (nodeTag(node))
{
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelEstimate, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ *pscan_size = heap_parallelscan_estimate(estate->es_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, *pscan_size);
+
+ /* key for paratial scan information. */
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ return true;
+ }
default:
break;
}
@@ -311,6 +330,8 @@ bool
ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
Size *pscan_size)
{
+ ParallelHeapScanDesc pscan;
+
if (node == NULL)
return false;
@@ -320,6 +341,25 @@ ExecParallelInitializeDSM(Node *node, ParallelContext *pcxt,
*/
switch (nodeTag(node))
{
+ case T_ResultState:
+ {
+ PlanState *planstate = ((ResultState*)node)->ps.lefttree;
+
+ return planstate_tree_walker((Node*)planstate, pcxt,
+ ExecParallelInitializeDSM, pscan_size);
+ }
+ case T_PartialSeqScanState:
+ {
+ EState *estate = ((PartialSeqScanState*)node)->ss.ps.state;
+
+ /* Store parallel heap scan descriptor in dynamic shared memory. */
+ pscan = shm_toc_allocate(pcxt->toc, *pscan_size);
+ heap_parallelscan_initialize(pscan,
+ ((PartialSeqScanState*)node)->ss.ss_currentRelation,
+ estate->es_snapshot);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_SCAN, pscan);
+ return true;
+ }
default:
break;
}
diff --git a/src/backend/executor/execProcnode.c b/src/backend/executor/execProcnode.c
index c181bf2..e24a439 100644
--- a/src/backend/executor/execProcnode.c
+++ b/src/backend/executor/execProcnode.c
@@ -100,6 +100,7 @@
#include "executor/nodeMergejoin.h"
#include "executor/nodeModifyTable.h"
#include "executor/nodeNestloop.h"
+#include "executor/nodePartialSeqscan.h"
#include "executor/nodeFunnel.h"
#include "executor/nodeRecursiveunion.h"
#include "executor/nodeResult.h"
@@ -197,6 +198,11 @@ ExecInitNode(Plan *node, EState *estate, int eflags)
estate, eflags);
break;
+ case T_PartialSeqScan:
+ result = (PlanState *) ExecInitPartialSeqScan((PartialSeqScan *) node,
+ estate, eflags);
+ break;
+
case T_Funnel:
result = (PlanState *) ExecInitFunnel((Funnel *) node,
estate, eflags);
@@ -422,6 +428,10 @@ ExecProcNode(PlanState *node)
result = ExecSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ result = ExecPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
case T_FunnelState:
result = ExecFunnel((FunnelState *) node);
break;
@@ -668,6 +678,10 @@ ExecEndNode(PlanState *node)
ExecEndSampleScan((SampleScanState *) node);
break;
+ case T_PartialSeqScanState:
+ ExecEndPartialSeqScan((PartialSeqScanState *) node);
+ break;
+
case T_FunnelState:
ExecEndFunnel((FunnelState *) node);
break;
diff --git a/src/backend/executor/nodePartialSeqscan.c b/src/backend/executor/nodePartialSeqscan.c
new file mode 100644
index 0000000..c18dce0
--- /dev/null
+++ b/src/backend/executor/nodePartialSeqscan.c
@@ -0,0 +1,308 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.c
+ * Support routines for partial sequential scans of relations.
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/executor/nodePartialSeqscan.c
+ *
+ *-------------------------------------------------------------------------
+ */
+/*
+ * INTERFACE ROUTINES
+ * ExecPartialSeqScan scans a relation partially.
+ * PartialSeqNext retrieve next tuple from heap.
+ * ExecInitPartialSeqScan creates and initializes a partial seqscan node.
+ * ExecEndPartialSeqScan releases any storage allocated.
+ */
+#include "postgres.h"
+
+#include "access/relscan.h"
+#include "executor/execdebug.h"
+#include "executor/execParallel.h"
+#include "executor/nodePartialSeqscan.h"
+#include "utils/rel.h"
+
+
+
+/* ----------------------------------------------------------------
+ * Scan Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * PartialSeqNext
+ *
+ * This is a workhorse for ExecPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+static TupleTableSlot *
+PartialSeqNext(PartialSeqScanState *node)
+{
+ HeapTuple tuple;
+ HeapScanDesc scandesc;
+ EState *estate;
+ ScanDirection direction;
+ TupleTableSlot *slot;
+
+ /*
+ * get information from the estate and scan state
+ */
+ scandesc = node->ss.ss_currentScanDesc;
+ estate = node->ss.ps.state;
+ direction = estate->es_direction;
+ slot = node->ss.ss_ScanTupleSlot;
+
+ /*
+ * get the next tuple from the table
+ */
+ tuple = heap_getnext(scandesc, direction);
+
+ /*
+ * save the tuple and the buffer returned to us by the access methods in
+ * our scan tuple slot and return the slot. Note: we pass 'false' because
+ * tuples returned by heap_getnext() are pointers onto disk pages and were
+ * not created with palloc() and so should not be pfree()'d. Note also
+ * that ExecStoreTuple will increment the refcount of the buffer; the
+ * refcount will not be dropped until the tuple table slot is cleared.
+ */
+ if (tuple)
+ ExecStoreTuple(tuple, /* tuple to store */
+ slot, /* slot to store in */
+ scandesc->rs_cbuf, /* buffer associated with this
+ * tuple */
+ false); /* don't pfree this pointer */
+ else
+ ExecClearTuple(slot);
+
+ return slot;
+}
+
+/*
+ * PartialSeqRecheck -- access method routine to recheck a tuple in EvalPlanQual
+ */
+static bool
+PartialSeqRecheck(PartialSeqScanState *node, TupleTableSlot *slot)
+{
+ /*
+ * Note that unlike IndexScan, PartialSeqScan never use keys in
+ * heap_beginscan (and this is very bad) - so, here we do not
+ * check are keys ok or not.
+ */
+ return true;
+}
+
+/* ----------------------------------------------------------------
+ * InitPartialScanRelation
+ *
+ * Set up to access the scan relation.
+ * ----------------------------------------------------------------
+ */
+static void
+InitPartialScanRelation(PartialSeqScanState *node, EState *estate, int eflags)
+{
+ Relation currentRelation;
+ shm_toc *toc;
+
+ /*
+ * get the relation object id from the relid'th entry in the range table,
+ * open that relation and acquire appropriate lock on it.
+ */
+ currentRelation = ExecOpenScanRelation(estate,
+ ((Scan *) node->ss.ps.plan)->scanrelid,
+ eflags);
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend and parallel workers retrieve it
+ * from shared memory. We set 'toc' (place to lookup parallel scan
+ * descriptor) as retrievied by attaching to dsm for parallel workers
+ * whereas master backend stores it directly in partial scan state node
+ * after initializing workers.
+ */
+ toc = GetParallelShmToc();
+ if (toc)
+ node->ss.ps.toc = toc;
+
+ node->ss.ss_currentRelation = currentRelation;
+
+ /* and report the scan tuple slot's rowtype */
+ ExecAssignScanType(&node->ss, RelationGetDescr(currentRelation));
+}
+
+/* ----------------------------------------------------------------
+ * ExecInitPartialSeqScan
+ * ----------------------------------------------------------------
+ */
+PartialSeqScanState *
+ExecInitPartialSeqScan(PartialSeqScan *node, EState *estate, int eflags)
+{
+ PartialSeqScanState *scanstate;
+
+ /*
+ * Once upon a time it was possible to have an outerPlan of a SeqScan, but
+ * not any more.
+ */
+ Assert(outerPlan(node) == NULL);
+ Assert(innerPlan(node) == NULL);
+
+ /*
+ * create state structure
+ */
+ scanstate = makeNode(PartialSeqScanState);
+ scanstate->ss.ps.plan = (Plan *) node;
+ scanstate->ss.ps.state = estate;
+
+ /*
+ * Miscellaneous initialization
+ *
+ * create expression context for node
+ */
+ ExecAssignExprContext(estate, &scanstate->ss.ps);
+
+ /*
+ * initialize child expressions
+ */
+ scanstate->ss.ps.targetlist = (List *)
+ ExecInitExpr((Expr *) node->plan.targetlist,
+ (PlanState *) scanstate);
+ scanstate->ss.ps.qual = (List *)
+ ExecInitExpr((Expr *) node->plan.qual,
+ (PlanState *) scanstate);
+
+ /*
+ * tuple table initialization
+ */
+ ExecInitResultTupleSlot(estate, &scanstate->ss.ps);
+ ExecInitScanTupleSlot(estate, &scanstate->ss);
+
+ /*
+ * initialize scan relation
+ */
+ InitPartialScanRelation(scanstate, estate, eflags);
+
+ scanstate->ss.ps.ps_TupFromTlist = false;
+
+ /*
+ * Initialize result tuple type and projection info.
+ */
+ ExecAssignResultTypeFromTL(&scanstate->ss.ps);
+ ExecAssignScanProjectionInfo(&scanstate->ss);
+
+ return scanstate;
+}
+
+/* ----------------------------------------------------------------
+ * ExecPartialSeqScan(node)
+ *
+ * Scans the relation and returns the next qualifying tuple.
+ * We call the ExecScan() routine and pass it the appropriate
+ * access method functions.
+ * ----------------------------------------------------------------
+ */
+TupleTableSlot *
+ExecPartialSeqScan(PartialSeqScanState *node)
+{
+ /*
+ * Initialize the scan on first execution, normally we initialize
+ * it during ExecutorStart phase, however we need ParallelHeapScanDesc
+ * to initialize the scan in case of this node and the same is
+ * initialized by the Funnel node during ExecutorRun phase.
+ */
+ if (!node->scan_initialized)
+ {
+ ParallelHeapScanDesc pscan;
+
+ /*
+ * Parallel scan descriptor is initialized and stored in dynamic shared
+ * memory segment by master backend, parallel workers and local scan by
+ * master backend retrieve it from shared memory. If the scan descriptor
+ * is available on first execution, then we need to re-initialize for
+ * rescan.
+ */
+ Assert(node->ss.ps.toc);
+
+ pscan = shm_toc_lookup(node->ss.ps.toc, PARALLEL_KEY_SCAN);
+
+ if (!node->ss.ss_currentScanDesc)
+ {
+ node->ss.ss_currentScanDesc =
+ heap_beginscan_parallel(node->ss.ss_currentRelation, pscan);
+ }
+ else
+ {
+ heap_parallel_rescan(pscan, node->ss.ss_currentScanDesc);
+ }
+
+ node->scan_initialized = true;
+ }
+
+ return ExecScan((ScanState *) node,
+ (ExecScanAccessMtd) PartialSeqNext,
+ (ExecScanRecheckMtd) PartialSeqRecheck);
+}
+
+/* ----------------------------------------------------------------
+ * ExecEndPartialSeqScan
+ *
+ * frees any storage allocated through C routines.
+ * ----------------------------------------------------------------
+ */
+void
+ExecEndPartialSeqScan(PartialSeqScanState *node)
+{
+ Relation relation;
+ HeapScanDesc scanDesc;
+
+ /*
+ * get information from node
+ */
+ relation = node->ss.ss_currentRelation;
+ scanDesc = node->ss.ss_currentScanDesc;
+
+ /*
+ * Free the exprcontext
+ */
+ ExecFreeExprContext(&node->ss.ps);
+
+ /*
+ * clean out the tuple table
+ */
+ ExecClearTuple(node->ss.ps.ps_ResultTupleSlot);
+ ExecClearTuple(node->ss.ss_ScanTupleSlot);
+
+ /*
+ * close heap scan
+ */
+ if (scanDesc)
+ heap_endscan(scanDesc);
+
+ /*
+ * close the heap relation.
+ */
+ ExecCloseScanRelation(relation);
+}
+
+/* ----------------------------------------------------------------
+ * Join Support
+ * ----------------------------------------------------------------
+ */
+
+/* ----------------------------------------------------------------
+ * ExecReScanPartialSeqScan
+ *
+ * Rescans the relation.
+ * ----------------------------------------------------------------
+ */
+void
+ExecReScanPartialSeqScan(PartialSeqScanState *node)
+{
+ if (node->scan_initialized)
+ node->scan_initialized = false;
+
+ ExecScanReScan((ScanState *) node);
+}
diff --git a/src/backend/executor/nodeResult.c b/src/backend/executor/nodeResult.c
index 8d3dde0..b348bfd 100644
--- a/src/backend/executor/nodeResult.c
+++ b/src/backend/executor/nodeResult.c
@@ -75,6 +75,13 @@ ExecResult(ResultState *node)
econtext = node->ps.ps_ExprContext;
/*
+ * Result node can be added as a gating node on top of PartialSeqScan
+ * node, so need to percolate toc information to outer node.
+ */
+ if (node->ps.toc)
+ outerPlanState(node)->toc = node->ps.toc;
+
+ /*
* check constant qualifications like (2 > 1), if not already done
*/
if (node->rs_checkqual)
diff --git a/src/backend/nodes/copyfuncs.c b/src/backend/nodes/copyfuncs.c
index 11d8191..afed75e 100644
--- a/src/backend/nodes/copyfuncs.c
+++ b/src/backend/nodes/copyfuncs.c
@@ -382,6 +382,22 @@ _copySampleScan(const SampleScan *from)
}
/*
+ * _copyPartialSeqScan
+ */
+static PartialSeqScan *
+_copyPartialSeqScan(const SeqScan *from)
+{
+ PartialSeqScan *newnode = makeNode(PartialSeqScan);
+
+ /*
+ * copy node superclass fields
+ */
+ CopyScanFields((const Scan *) from, (Scan *) newnode);
+
+ return newnode;
+}
+
+/*
* _copyFunnel
*/
static Funnel *
@@ -4260,6 +4276,9 @@ copyObject(const void *from)
case T_SampleScan:
retval = _copySampleScan(from);
break;
+ case T_PartialSeqScan:
+ retval = _copyPartialSeqScan(from);
+ break;
case T_Funnel:
retval = _copyFunnel(from);
break;
diff --git a/src/backend/nodes/outfuncs.c b/src/backend/nodes/outfuncs.c
index 232b950..2c66490 100644
--- a/src/backend/nodes/outfuncs.c
+++ b/src/backend/nodes/outfuncs.c
@@ -456,6 +456,14 @@ _outSampleScan(StringInfo str, const SampleScan *node)
}
static void
+_outPartialSeqScan(StringInfo str, const SeqScan *node)
+{
+ WRITE_NODE_TYPE("PARTIALSEQSCAN");
+
+ _outScanInfo(str, (const Scan *) node);
+}
+
+static void
_outFunnel(StringInfo str, const Funnel *node)
{
WRITE_NODE_TYPE("FUNNEL");
@@ -3015,6 +3023,9 @@ _outNode(StringInfo str, const void *obj)
case T_SampleScan:
_outSampleScan(str, obj);
break;
+ case T_PartialSeqScan:
+ _outPartialSeqScan(str, obj);
+ break;
case T_Funnel:
_outFunnel(str, obj);
break;
diff --git a/src/backend/nodes/readfuncs.c b/src/backend/nodes/readfuncs.c
index e0fe8d5..e340543 100644
--- a/src/backend/nodes/readfuncs.c
+++ b/src/backend/nodes/readfuncs.c
@@ -1412,6 +1412,81 @@ _readPlannedStmt(void)
}
/*
+ * _readPlan
+ */
+static Plan *
+_readPlan(void)
+{
+ READ_LOCALS(Plan);
+
+ READ_FLOAT_FIELD(startup_cost);
+ READ_FLOAT_FIELD(total_cost);
+ READ_FLOAT_FIELD(plan_rows);
+ READ_INT_FIELD(plan_width);
+ READ_NODE_FIELD(targetlist);
+ READ_NODE_FIELD(qual);
+ READ_NODE_FIELD(lefttree);
+ READ_NODE_FIELD(righttree);
+ READ_NODE_FIELD(initPlan);
+ READ_BITMAPSET_FIELD(extParam);
+ READ_BITMAPSET_FIELD(allParam);
+
+ READ_DONE();
+}
+
+/*
+ * _readScan
+ */
+static Scan *
+_readScan(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(PartialSeqScan);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_UINT_FIELD(scanrelid);
+
+ READ_DONE();
+}
+
+/*
+ * _readResult
+ */
+static Result *
+_readResult(void)
+{
+ Plan *local_plan;
+ READ_LOCALS(Result);
+
+ local_plan = _readPlan();
+ local_node->plan.startup_cost = local_plan->startup_cost;
+ local_node->plan.total_cost = local_plan->total_cost;
+ local_node->plan.plan_rows = local_plan->plan_rows;
+ local_node->plan.plan_width = local_plan->plan_width;
+ local_node->plan.targetlist = local_plan->targetlist;
+ local_node->plan.qual = local_plan->qual;
+ local_node->plan.lefttree = local_plan->lefttree;
+ local_node->plan.righttree = local_plan->righttree;
+ local_node->plan.initPlan = local_plan->initPlan;
+ local_node->plan.extParam = local_plan->extParam;
+ local_node->plan.allParam = local_plan->allParam;
+ READ_NODE_FIELD(resconstantqual);
+
+ READ_DONE();
+}
+
+/*
* parseNodeString
*
* Given a character string representing a node tree, parseNodeString creates
@@ -1553,6 +1628,10 @@ parseNodeString(void)
return_value = _readPlanInvalItem();
else if (MATCH("PLANNEDSTMT", 11))
return_value = _readPlannedStmt();
+ else if (MATCH("PARTIALSEQSCAN", 14))
+ return_value = _readScan();
+ else if (MATCH("RESULT", 6))
+ return_value = _readResult();
else
{
elog(ERROR, "badly formatted node string \"%.32s\"...", token);
diff --git a/src/backend/optimizer/path/Makefile b/src/backend/optimizer/path/Makefile
index 6864a62..6e462b1 100644
--- a/src/backend/optimizer/path/Makefile
+++ b/src/backend/optimizer/path/Makefile
@@ -13,6 +13,6 @@ top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
OBJS = allpaths.o clausesel.o costsize.o equivclass.o indxpath.o \
- joinpath.o joinrels.o pathkeys.o tidpath.o
+ joinpath.o joinrels.o pathkeys.o parallelpath.o tidpath.o
include $(top_srcdir)/src/backend/common.mk
diff --git a/src/backend/optimizer/path/allpaths.c b/src/backend/optimizer/path/allpaths.c
index 8fc1cfd..c2ae95d 100644
--- a/src/backend/optimizer/path/allpaths.c
+++ b/src/backend/optimizer/path/allpaths.c
@@ -477,6 +477,9 @@ set_plain_rel_pathlist(PlannerInfo *root, RelOptInfo *rel, RangeTblEntry *rte)
/* Consider sequential scan */
add_path(rel, create_seqscan_path(root, rel, required_outer));
+ /* Consider parallel scans */
+ create_parallelscan_paths(root, rel, required_outer);
+
/* Consider index scans */
create_index_paths(root, rel);
diff --git a/src/backend/optimizer/path/costsize.c b/src/backend/optimizer/path/costsize.c
index 78d976a..55da0c2 100644
--- a/src/backend/optimizer/path/costsize.c
+++ b/src/backend/optimizer/path/costsize.c
@@ -296,6 +296,50 @@ cost_samplescan(Path *path, PlannerInfo *root,
}
/*
+ * cost_patialseqscan
+ * Determines and returns the cost of scanning a relation partially.
+ *
+ * 'baserel' is the relation to be scanned
+ * 'param_info' is the ParamPathInfo if this is a parameterized path, else NULL
+ * 'nworkers' are the number of workers among which the work will be
+ * distributed
+ */
+void
+cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers)
+{
+ Cost startup_cost = 0;
+ Cost run_cost = 0;
+
+ cost_seqscan(path, root, baserel, param_info);
+
+ startup_cost = path->startup_cost;
+
+ run_cost = path->total_cost - startup_cost;
+
+ /*
+ * Account for small cost for communication related to scan
+ * via the ParallelHeapScanDesc.
+ */
+ run_cost += 0.01;
+
+ /*
+ * Runtime cost will be equally shared by all workers.
+ * Here assumption is that disk access cost will also be
+ * equally shared between workers which is generally true
+ * unless there are too many workers working on a relatively
+ * lesser number of blocks. If we come across any such case,
+ * then we can think of changing the current cost model for
+ * partial sequiantial scan.
+ */
+ run_cost = run_cost / (nworkers + 1);
+
+ path->startup_cost = startup_cost;
+ path->total_cost = startup_cost + run_cost;
+}
+
+/*
* cost_funnel
* Determines and returns the cost of funnel path.
*
diff --git a/src/backend/optimizer/path/parallelpath.c b/src/backend/optimizer/path/parallelpath.c
new file mode 100644
index 0000000..e813ba1
--- /dev/null
+++ b/src/backend/optimizer/path/parallelpath.c
@@ -0,0 +1,93 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallelpath.c
+ * Routines to determine parallel paths for scanning a given relation.
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ *
+ * IDENTIFICATION
+ * src/backend/optimizer/path/parallelpath.c
+ *
+ *-------------------------------------------------------------------------
+ */
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "optimizer/cost.h"
+#include "optimizer/pathnode.h"
+#include "optimizer/paths.h"
+#include "parser/parsetree.h"
+#include "utils/rel.h"
+
+
+/*
+ * create_parallelscan_paths
+ * Create paths corresponding to parallel scans of the given rel.
+ * Currently we only support partial sequential scan.
+ *
+ * Candidate paths are added to the rel's pathlist (using add_path).
+ */
+void
+create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer)
+{
+ int num_parallel_workers = 0;
+ int estimated_parallel_workers = 0;
+ Oid reloid;
+ Relation relation;
+ Path *subpath;
+
+ /*
+ * parallel scan is possible only if user has set parallel_seqscan_degree
+ * to value greater than 0 and the query is parallel-safe.
+ */
+ if (parallel_seqscan_degree <= 0 || !root->glob->parallelModeOK)
+ return;
+
+ /*
+ * There should be atleast thousand pages to scan for each worker.
+ * This number is somewhat arbitratry, however we don't want to
+ * spawn workers to scan smaller relations as that will be costly.
+ */
+ estimated_parallel_workers = rel->pages / 1000;
+
+ if (estimated_parallel_workers <= 0)
+ return;
+
+ reloid = planner_rt_fetch(rel->relid, root)->relid;
+
+ relation = heap_open(reloid, NoLock);
+
+ /*
+ * Temporary relations can't be scanned by parallel workers as
+ * they are visible only to local sessions.
+ */
+ if (RelationUsesLocalBuffers(relation))
+ {
+ heap_close(relation, NoLock);
+ return;
+ }
+
+ heap_close(relation, NoLock);
+
+ 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;
+
+ /*
+ * Create the partial scan path which each worker backend needs to
+ * execute.
+ */
+ subpath = create_partialseqscan_path(root, rel, required_outer,
+ num_parallel_workers);
+
+ /* Create the funnel path which master backend needs to execute. */
+ add_path(rel, (Path *) create_funnel_path(root, rel, subpath,
+ required_outer,
+ num_parallel_workers));
+}
diff --git a/src/backend/optimizer/plan/createplan.c b/src/backend/optimizer/plan/createplan.c
index 68d8837..6c95341 100644
--- a/src/backend/optimizer/plan/createplan.c
+++ b/src/backend/optimizer/plan/createplan.c
@@ -60,6 +60,8 @@ static SeqScan *create_seqscan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
static SampleScan *create_samplescan_plan(PlannerInfo *root, Path *best_path,
List *tlist, List *scan_clauses);
+static Scan *create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses);
static Funnel *create_funnel_plan(PlannerInfo *root,
FunnelPath *best_path);
static Scan *create_indexscan_plan(PlannerInfo *root, IndexPath *best_path,
@@ -106,6 +108,8 @@ static void copy_plan_costsize(Plan *dest, Plan *src);
static SeqScan *make_seqscan(List *qptlist, List *qpqual, Index scanrelid);
static SampleScan *make_samplescan(List *qptlist, List *qpqual, Index scanrelid,
TableSampleClause *tsc);
+static PartialSeqScan *make_partialseqscan(List *qptlist, List *qpqual,
+ Index scanrelid);
static Funnel *make_funnel(List *qptlist, List *qpqual,
Index scanrelid, int nworkers,
Plan *subplan);
@@ -239,6 +243,7 @@ create_plan_recurse(PlannerInfo *root, Path *best_path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_IndexScan:
case T_IndexOnlyScan:
case T_BitmapHeapScan:
@@ -365,6 +370,13 @@ create_scan_plan(PlannerInfo *root, Path *best_path)
scan_clauses);
break;
+ case T_PartialSeqScan:
+ plan = (Plan *) create_partialseqscan_plan(root,
+ best_path,
+ tlist,
+ scan_clauses);
+ break;
+
case T_IndexScan:
plan = (Plan *) create_indexscan_plan(root,
(IndexPath *) best_path,
@@ -569,6 +581,7 @@ disuse_physical_tlist(PlannerInfo *root, Plan *plan, Path *path)
{
case T_SeqScan:
case T_SampleScan:
+ case T_PartialSeqScan:
case T_Funnel:
case T_IndexScan:
case T_IndexOnlyScan:
@@ -1204,6 +1217,46 @@ create_samplescan_plan(PlannerInfo *root, Path *best_path,
}
/*
+ * create_partialseqscan_plan
+ *
+ * Returns a partial seqscan plan for the base relation scanned by
+ * 'best_path' with restriction clauses 'scan_clauses' and targetlist
+ * 'tlist'.
+ */
+static Scan *
+create_partialseqscan_plan(PlannerInfo *root, Path *best_path,
+ List *tlist, List *scan_clauses)
+{
+ Scan *scan_plan;
+ Index scan_relid = best_path->parent->relid;
+
+ /* it should be a base rel... */
+ Assert(scan_relid > 0);
+ Assert(best_path->parent->rtekind == RTE_RELATION);
+
+ /* Sort clauses into best execution order */
+ scan_clauses = order_qual_clauses(root, scan_clauses);
+
+ /* Reduce RestrictInfo list to bare expressions; ignore pseudoconstants */
+ scan_clauses = extract_actual_clauses(scan_clauses, false);
+
+ /* Replace any outer-relation variables with nestloop params */
+ if (best_path->param_info)
+ {
+ scan_clauses = (List *)
+ replace_nestloop_params(root, (Node *) scan_clauses);
+ }
+
+ scan_plan = (Scan *) make_partialseqscan(tlist,
+ scan_clauses,
+ scan_relid);
+
+ copy_path_costsize(&scan_plan->plan, best_path);
+
+ return scan_plan;
+}
+
+/*
* create_funnel_plan
*
* Returns a funnel plan for the base relation scanned by
@@ -3533,6 +3586,24 @@ make_samplescan(List *qptlist,
return node;
}
+static PartialSeqScan *
+make_partialseqscan(List *qptlist,
+ List *qpqual,
+ Index scanrelid)
+{
+ PartialSeqScan *node = makeNode(PartialSeqScan);
+ Plan *plan = &node->plan;
+
+ /* cost should be inserted by caller */
+ plan->targetlist = qptlist;
+ plan->qual = qpqual;
+ plan->lefttree = NULL;
+ plan->righttree = NULL;
+ node->scanrelid = scanrelid;
+
+ return node;
+}
+
static Funnel *
make_funnel(List *qptlist,
List *qpqual,
diff --git a/src/backend/optimizer/plan/setrefs.c b/src/backend/optimizer/plan/setrefs.c
index 12f6635..39c35c6 100644
--- a/src/backend/optimizer/plan/setrefs.c
+++ b/src/backend/optimizer/plan/setrefs.c
@@ -442,6 +442,7 @@ set_plan_refs(PlannerInfo *root, Plan *plan, int rtoffset)
switch (nodeTag(plan))
{
case T_SeqScan:
+ case T_PartialSeqScan:
{
SeqScan *splan = (SeqScan *) plan;
@@ -2308,6 +2309,11 @@ fix_node_funcids(Plan *node)
switch (nodeTag(node))
{
+ case T_Result:
+ fix_opfuncids((Node*) (((Result *)node)->resconstantqual));
+ break;
+ case T_PartialSeqScan:
+ break;
default:
elog(ERROR, "unrecognized node type: %d", (int) nodeTag(node));
break;
diff --git a/src/backend/optimizer/plan/subselect.c b/src/backend/optimizer/plan/subselect.c
index 073a7f5..37b5909 100644
--- a/src/backend/optimizer/plan/subselect.c
+++ b/src/backend/optimizer/plan/subselect.c
@@ -2243,6 +2243,7 @@ finalize_plan(PlannerInfo *root, Plan *plan, Bitmapset *valid_params,
context.paramids = bms_add_members(context.paramids, scan_params);
break;
+ case T_PartialSeqScan:
case T_Funnel:
context.paramids = bms_add_members(context.paramids, scan_params);
break;
diff --git a/src/backend/optimizer/util/pathnode.c b/src/backend/optimizer/util/pathnode.c
index 276ad96..ef2725a 100644
--- a/src/backend/optimizer/util/pathnode.c
+++ b/src/backend/optimizer/util/pathnode.c
@@ -732,6 +732,28 @@ create_samplescan_path(PlannerInfo *root, RelOptInfo *rel, Relids required_outer
}
/*
+ * create_partialseqscan_path
+ * Creates a path corresponding to a partial sequential scan, returning the
+ * pathnode.
+ */
+Path *
+create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers)
+{
+ Path *pathnode = makeNode(Path);
+
+ pathnode->pathtype = T_PartialSeqScan;
+ pathnode->parent = rel;
+ pathnode->param_info = get_baserel_parampathinfo(root, rel,
+ required_outer);
+ pathnode->pathkeys = NIL; /* partialseqscan has unordered result */
+
+ cost_patialseqscan(pathnode, root, rel, pathnode->param_info, nworkers);
+
+ return pathnode;
+}
+
+/*
* create_funnel_path
*
* Creates a path corresponding to a funnel scan, returning the
diff --git a/src/include/access/heapam.h b/src/include/access/heapam.h
index b3e3202..ead8411 100644
--- a/src/include/access/heapam.h
+++ b/src/include/access/heapam.h
@@ -96,8 +96,9 @@ extern Relation heap_openrv_extended(const RangeVar *relation,
#define heap_close(r,l) relation_close(r,l)
-/* struct definition appears in relscan.h */
+/* struct definitions appear in relscan.h */
typedef struct HeapScanDescData *HeapScanDesc;
+typedef struct ParallelHeapScanDescData *ParallelHeapScanDesc;
/*
* HeapScanIsValid
@@ -121,11 +122,16 @@ extern void heap_setscanlimits(HeapScanDesc scan, BlockNumber startBlk,
BlockNumber endBlk);
extern void heapgetpage(HeapScanDesc scan, BlockNumber page);
extern void heap_rescan(HeapScanDesc scan, ScanKey key);
+extern void heap_parallel_rescan(ParallelHeapScanDesc pscan, HeapScanDesc scan);
extern void heap_rescan_set_params(HeapScanDesc scan, ScanKey key,
bool allow_strat, bool allow_sync, bool allow_pagemode);
extern void heap_endscan(HeapScanDesc scan);
extern HeapTuple heap_getnext(HeapScanDesc scan, ScanDirection direction);
+extern Size heap_parallelscan_estimate(Snapshot snapshot);
+extern void heap_parallelscan_initialize(ParallelHeapScanDesc target,
+ Relation relation, Snapshot snapshot);
+extern HeapScanDesc heap_beginscan_parallel(Relation, ParallelHeapScanDesc);
extern bool heap_fetch(Relation relation, Snapshot snapshot,
HeapTuple tuple, Buffer *userbuf, bool keep_buf,
diff --git a/src/include/access/relscan.h b/src/include/access/relscan.h
index 6e62319..f962f83 100644
--- a/src/include/access/relscan.h
+++ b/src/include/access/relscan.h
@@ -20,6 +20,17 @@
#include "access/itup.h"
#include "access/tupdesc.h"
+/* Struct for parallel scan setup */
+typedef struct ParallelHeapScanDescData
+{
+ Oid phs_relid;
+ BlockNumber phs_nblocks;
+ slock_t phs_mutex;
+ BlockNumber phs_cblock;
+ BlockNumber phs_startblock;
+ bool phs_firstpass;
+ char phs_snapshot_data[FLEXIBLE_ARRAY_MEMBER];
+} ParallelHeapScanDescData;
typedef struct HeapScanDescData
{
@@ -49,6 +60,7 @@ typedef struct HeapScanDescData
BlockNumber rs_cblock; /* current block # in scan, if any */
Buffer rs_cbuf; /* current buffer in scan, if any */
/* NB: if rs_cbuf is not InvalidBuffer, we hold a pin on that buffer */
+ ParallelHeapScanDesc rs_parallel; /* parallel scan information */
/* these fields only used in page-at-a-time mode and for bitmap scans */
int rs_cindex; /* current tuple's index in vistuples */
diff --git a/src/include/executor/nodePartialSeqscan.h b/src/include/executor/nodePartialSeqscan.h
new file mode 100644
index 0000000..f97c706
--- /dev/null
+++ b/src/include/executor/nodePartialSeqscan.h
@@ -0,0 +1,25 @@
+/*-------------------------------------------------------------------------
+ *
+ * nodePartialSeqscan.h
+ * prototypes for nodePartialSeqscan.c
+ *
+ *
+ * Portions Copyright (c) 1996-2015, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/executor/nodePartialSeqscan.h
+ *
+ *-------------------------------------------------------------------------
+ */
+#ifndef NODEPARTIALSEQSCAN_H
+#define NODEPARTIALSEQSCAN_H
+
+#include "nodes/execnodes.h"
+
+extern PartialSeqScanState *ExecInitPartialSeqScan(PartialSeqScan *node,
+ EState *estate, int eflags);
+extern TupleTableSlot *ExecPartialSeqScan(PartialSeqScanState *node);
+extern void ExecEndPartialSeqScan(PartialSeqScanState *node);
+extern void ExecReScanPartialSeqScan(PartialSeqScanState *node);
+
+#endif /* NODEPARTIALSEQSCAN_H */
diff --git a/src/include/nodes/execnodes.h b/src/include/nodes/execnodes.h
index 8f10c4e..9cb31b5 100644
--- a/src/include/nodes/execnodes.h
+++ b/src/include/nodes/execnodes.h
@@ -1292,6 +1292,16 @@ typedef struct SampleScanState
} SampleScanState;
/*
+ * PartialSeqScanState extends ScanState by storing additional information
+ * related to scan.
+ */
+typedef struct PartialSeqScanState
+{
+ ScanState ss; /* its first field is NodeTag */
+ bool scan_initialized; /* used to determine if the scan is initialized */
+} PartialSeqScanState;
+
+/*
* FunnelState extends ScanState by storing additional information
* related to parallel workers.
* pcxt parallel context for managing generic state information
diff --git a/src/include/nodes/nodes.h b/src/include/nodes/nodes.h
index f456004..bd87a84 100644
--- a/src/include/nodes/nodes.h
+++ b/src/include/nodes/nodes.h
@@ -52,6 +52,7 @@ typedef enum NodeTag
T_Scan,
T_SeqScan,
T_SampleScan,
+ T_PartialSeqScan,
T_Funnel,
T_IndexScan,
T_IndexOnlyScan,
@@ -100,6 +101,7 @@ typedef enum NodeTag
T_ScanState,
T_SeqScanState,
T_SampleScanState,
+ T_PartialSeqScanState,
T_FunnelState,
T_IndexScanState,
T_IndexOnlyScanState,
diff --git a/src/include/nodes/plannodes.h b/src/include/nodes/plannodes.h
index 69302af..2d25a01 100644
--- a/src/include/nodes/plannodes.h
+++ b/src/include/nodes/plannodes.h
@@ -296,6 +296,12 @@ typedef struct SampleScan
struct TableSampleClause *tablesample;
} SampleScan;
+/* ----------------
+ * partial sequential scan node
+ * ----------------
+ */
+typedef SeqScan PartialSeqScan;
+
/* ------------
* Funnel node
* ------------
diff --git a/src/include/optimizer/cost.h b/src/include/optimizer/cost.h
index 994ea83..7592560 100644
--- a/src/include/optimizer/cost.h
+++ b/src/include/optimizer/cost.h
@@ -80,6 +80,9 @@ extern void cost_seqscan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
extern void cost_samplescan(Path *path, PlannerInfo *root, RelOptInfo *baserel,
ParamPathInfo *param_info);
+extern void cost_patialseqscan(Path *path, PlannerInfo *root,
+ RelOptInfo *baserel, ParamPathInfo *param_info,
+ int nworkers);
extern void cost_funnel(FunnelPath *path, PlannerInfo *root,
RelOptInfo *baserel, ParamPathInfo *param_info);
extern void cost_index(IndexPath *path, PlannerInfo *root,
diff --git a/src/include/optimizer/pathnode.h b/src/include/optimizer/pathnode.h
index 9d31b93..a2b1f3d 100644
--- a/src/include/optimizer/pathnode.h
+++ b/src/include/optimizer/pathnode.h
@@ -34,6 +34,8 @@ extern Path *create_seqscan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
extern Path *create_samplescan_path(PlannerInfo *root, RelOptInfo *rel,
Relids required_outer);
+extern Path *create_partialseqscan_path(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer, int nworkers);
extern FunnelPath *create_funnel_path(PlannerInfo *root,
RelOptInfo *rel, Path *subpath, Relids required_outer,
int nworkers);
diff --git a/src/include/optimizer/paths.h b/src/include/optimizer/paths.h
index 87123a5..e7db9ab 100644
--- a/src/include/optimizer/paths.h
+++ b/src/include/optimizer/paths.h
@@ -55,6 +55,14 @@ extern void debug_print_rel(PlannerInfo *root, RelOptInfo *rel);
#endif
/*
+ * parallelpath.c
+ * routines to generate parallel scan paths
+ */
+
+extern void create_parallelscan_paths(PlannerInfo *root, RelOptInfo *rel,
+ Relids required_outer);
+
+/*
* indxpath.c
* routines to generate index paths
*/
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.12 rows=20000008 width=435)
Filter: (NOT (SubPlan 1))
Number of Workers: 16
-> Partial Seq Scan on test01 (cost=0.00..155114352184.12
rows=20000008 width=435)
Filter: (NOT (SubPlan 1))
SubPlan 1
-> Materialize (cost=0.00..130883.67 rows=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=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)
(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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Sep 9, 2015 at 8:09 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
Sounds sensible, will look into doing that way.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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, ScanDirection direction);
+
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.27 rows=5000000 width=214)
Filter: (SubPlan 1)
Number of Workers: 10
-> Partial Seq Scan on tbl a (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.00 rows=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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Mon, 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 17, 2015 at 6:29 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
wrote:
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.
As mentioned previously [1]/messages/by-id/CA+TgmobeqxZtP4crqtx36Mx7xtty-FsMFpuuRsVJOi8B6QRTGA@mail.gmail.com, 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]/messages/by-id/CA+TgmobeqxZtP4crqtx36Mx7xtty-FsMFpuuRsVJOi8B6QRTGA@mail.gmail.com) 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.
[1]: /messages/by-id/CA+TgmobeqxZtP4crqtx36Mx7xtty-FsMFpuuRsVJOi8B6QRTGA@mail.gmail.com
/messages/by-id/CA+TgmobeqxZtP4crqtx36Mx7xtty-FsMFpuuRsVJOi8B6QRTGA@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 17, 2015 at 1:40 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.
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.
CREATE TABLE t1(c1, c2) AS SELECT g, repeat('x', 5) FROM
generate_series(1, 10000000) g;
CREATE TABLE t2(c1, c2) AS SELECT g, repeat('x', 5) FROM
generate_series(1, 1000000) g;
set parallel_seqscan_degree=2;
set cpu_tuple_comm_cost=0.01;
explain select * from t1 where c1 not in (select c1 from t2 where c2 =
'xxxx');
QUERY PLAN
--------------------------------------------------------------------------------
----
Funnel on t1 (cost=11536.88..126809.17 rows=3432492 width=36)
Filter: (NOT (hashed SubPlan 1))
Number of Workers: 2
-> Partial Seq Scan on t1 (cost=11536.88..58159.32 rows=3432492
width=36)
Filter: (NOT (hashed SubPlan 1))
SubPlan 1
-> Funnel on t2 (cost=0.00..11528.30 rows=3433 width=4)
Filter: (c2 = 'xxxx'::text)
Number of Workers: 2
-> Partial Seq Scan on t2 (cost=0.00..4662.68 rows=3433
width
=4)
Filter: (c2 = 'xxxx'::text)
SubPlan 1
-> Funnel on t2 (cost=0.00..11528.30 rows=3433 width=4)
Filter: (c2 = 'xxxx'::text)
Number of Workers: 2
-> Partial Seq Scan on t2 (cost=0.00..4662.68 rows=3433
width=4)
Filter: (c2 = 'xxxx'::text)
(17 rows)
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?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Sep 15, 2015 at 8:34 AM, Haribabu Kommi <kommi.haribabu@gmail.com>
wrote:
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.
Thanks for the review.
+ 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.
makes sense, will change in next version of patch.
+ 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.
No, that's not true, it has projinfo for the cases where it is required.
So always it uses
the scan tuple slot. In case if it is different, we need to add the
ExecProject
call in ExecFunnel function.
It will not use Scan tuple slot for cases for column list contains
expression
or other cases where projection is required. Currently we don't need
separate
ExecProject as there is no case where workers won't do projection for us.
However in future we might need for something like restrictive functions.
I think it is better to add a comment explaining the same which I will do in
next version. Does that makes sense?
+ 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.
Yes, you are right. The main reason to keep it like this is that we
want to finish execution without going into error path, so that
the collected stats can be communicated back. I am not sure trying
to do better in this case is worth at this point.
+ 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.
I think we can remove this check as pointed out by Robert as well.
+ if (!parallelstmt->inst_options)
+ (*receiver->rDestroy) (receiver);Why only when there is no instruementation only, the receiver needs to
be destroyed?
No, receiver should be destroyed unconditionally. This is remnant of the
previous versions when receiver was created for no instrumentation case.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Thu, Sep 17, 2015 at 6:58 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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
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]/messages/by-id/CAA4eK1LNt6wQBCxKsMj_QC+GahBuwyKWsQn6UL3nWVQ2savzwg@mail.gmail.com.
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.
[1]: /messages/by-id/CAA4eK1LNt6wQBCxKsMj_QC+GahBuwyKWsQn6UL3nWVQ2savzwg@mail.gmail.com
/messages/by-id/CAA4eK1LNt6wQBCxKsMj_QC+GahBuwyKWsQn6UL3nWVQ2savzwg@mail.gmail.com
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
Attachments:
On Thu, Sep 17, 2015 at 2:28 AM, Robert Haas <robertmhaas@gmail.com> wrote:
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.
I think we can combine them and have one comment.
+GetParallelSupportInfo(shm_toc *toc, ParamListInfo *params,
Could this be a static function? Will it really be needed outside this
file?
It is already declared as static, but will add static in function definition
as well.
And is there any use case for letting some of the arguments be NULL?
In earlier versions of patch this API was used from other places, but now
there is no such use, so will change accordingly.
+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).
Will change according to recent commit for planstate_tree_walker
+ 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.
+ 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.
+ /* + * 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.
+/* + * _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.
We are not copying PlanInvalItems, so I don't think this is required.
Now I don't exactly remember why I have added this at first place,
one reason could be that in earlier versions PlanInvalItems might
have been copied. Anyway, I will verify it once more and if not
required, I will remove it.
+#parallel_setup_cost = 0.0 # same scale as above +#define DEFAULT_PARALLEL_SETUP_COST 0.0This value is probably a bit on the low side.
How about keeping it as 10.0?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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?
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.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Thu, Sep 17, 2015 at 4:44 PM, Robert Haas <robertmhaas@gmail.com> wrote:
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
support for all of them in outfuncs and needs to add for most of them
in readfuncs.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.0This 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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.00I 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
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
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
Attachments:
Parallel_seqscan_perf_test.xlsxapplication/vnd.openxmlformats-officedocument.spreadsheetml.sheet; name=Parallel_seqscan_perf_test.xlsxDownload
PK ! ��X�z [Content_Types].xml �(� �T�j�0��F�+I��'�.�6��kl����L���;vJ��i������e�y���*�%4�&�wE6u��<����������t�����`���W[LDA������Ja�<X>�\��g��W�\� ����L�%���C/��EI���o������^M��}iRE,T.�> ��,3)h�.*���P �����aDl�<���v�[W1W6��0��� �����m��#
�XzW{��R~�0�97����mM���R��t��o.�l^�+��5�-u�����?�A�s ���G��\ �����
�%�B��m��.�'�91��<r�h��]d���@��>4�-���#�=�A0B���n�d�� �� PK ! �U0#� L _rels/.rels �(� ���N�0��H�C���nH���LH�!T�$����$@����Jc�����?[���iTb/N��(A�3b{�jx��V�b"gi��aW��l_x���b���������#b4O��r��0Q�ah���e���=��P-<��j{�>�<���4Mox/�}b�N�@�;�v�Cf�����B�I������"c�&�\O���8q"K��H��<��s@���.�h����<����Md�a��T_ �� PK ! � �( � xl/_rels/workbook.xml.rels �(� ���j�0������}q�ne�:��A�[� &Q���6��'o?�C�@�.��$�}?�������j�U�%)���Z�(�8�><� ������`@�Cq��Ns�D��$��%������`�)qm��.��c�uy�
�<M�2���b�)���p�6 N����k��nK|q�g���X�o�d9��� +�Z$��&�� ���7���`��dK0�5a����;��B��j�^�yZ��.�~
�������O /�c)�w����b� �� PK ! x�*�v � xl/workbook.xml��]o� ����?������XM���Yv��r�rj�������ts1�v� ���t~(4� ��5 ��%`R+��%�m�����a���@B���|v}5���o����<�r��Os(����d�"`�v����9@(4�Q4f�P�����2���M�L8AhP��U��l�)
��#"��E���)���������5�h��\TJ��� P6�2����LT:��^G�����q���b�������6�H['�=vs>��n�%C�P�n8��A���"�x����M�hGbZk>����2t������K0nm���k"_���7Q8qK7�?1������ =��F�%��Q��Y�,:�����<�y��u?n� �� PK ! �b�m� � xl/theme/theme1.xml�YOo�6��w tom'�u�����M�n�i��XS�@�I}�����a���0l+��t�&[����HJ��K����D"|���#u�����C"$�q��]�z��>�8h{w��K�
�c�xL���H�����]��*$A�>�����J%����a��<!1�M����WT���U���f%�4�P�# {{2�>ACM�������J����&M�;��4B�e� t�Y�>c~4$���
&�^������
�L1�bma]���u���t���(gZ��[Wvr���2���u{���`�M�,E���F���,���2�n�Q�����%�[�N��Je�D
�>����f}{����7����v��t�d��%|�J�Yw�2O�����~J=�L8�-�o |���(��<�4� ���X��}.� �@����'d�}��.�F�b� o\��C�\����MT��0��z�����S���������t�����--g�.������~����?�~����xY����'���y92h!��/�������>����%�m�GE��FD�[��t3�q%'#q��Sg�v ��
��9fe�q�wW@�(^��wd�b�h �a� �8g.J
pC�*Xx8��r�bV�`|X���c���YU3J����8b�3+��(�������Q��u���K>Q�ELKM2�#'��vi~����vl�wu8+�z��HH�J����:�)����
~��L��\�E\O*�t@G�1��l�m��~C�*u��G.R(:-�ys^D��i7�QR��8,b?�SQ���*��q7C�;��+�}�����;4pDZ����_^'�����M01UJ�S�#�]�f�����l��m����g�D�^����<�� d����B����[_�W�����E)�*��k��;Z�xO(c5g��4���
h��A��:I~KBx��\ �Y�WQB�@�^�4�@���.��h�Kik<����6�b+��j�������9#U`���uM������DA��aV�B��[��f���-WY�������j0�[:�X� �~��;������Q����t����>�z/��f�����"Z��x��
�Z��p;�����+�e�{/e��P;��,.&'��Q�k5��q��&pT��(�K�Lb�}���
�S��d����L17 jp�a����S!���
3�� �5'+�Z�zQ
�T��I�����Ivt]K&����#�v�5-�|����#4b3q���:TA�1�p�a*�~��9mm3��4���bg�1KB��[��Y&[�)H��� �V*�Q������ U�a�?SE�'p�>���vX`�3��q�BU( ���8���W�0
Aw��� ����9K��5�$�
����P�e�D�)�j��eI�������2�b��!aC]�zo�P�n�IZ�d���i������d��ks���|l2�Rn6
Mf�\��=X��v�Y���EE�����gY�
[A+M����[��XK�52�����`�%p�����������7�!?������&aQ}�6HH;8����`���i��I[-��/�����0���,�>�����e���E;��ck;������)
C�� cc��?f��}p�|6�1%M0��*����<������� �� PK ! ;m2K� B # xl/worksheets/_rels/sheet1.xml.rels�����0E��Cx{���CS7"�U�b�������{�e���p��6��<�f�,����ch��{�-�A�8�� -<�a�.��NNJ���X��Q$��~���� ��>��I�y0������jm�_�/N��,�}W�:=RY���}<n���H���I9�`>�H9�E���bA�w��k}�m���� �� PK ! ?�T�E �? xl/worksheets/sheet2.xml�[�n�8}_`������H��$����� ��^�����Y��=��{x�*�����vZ<.�������~�m�6��=�v�������l��9<��_����W�S1���������}{?�}{��������8��^����O�����c�X�6����>��(y>��3�{|Y�>�������m��q���w�skay������n���o����9n��g�����8u��������_�}��9�?`���mw�������//��������P�Mg��g`~�����9���m�����\�����=��������c�l�t�x�3�g��q��=;>��}>��oo ���/��W
���~��i���h.fk|}�ZxSb�g����b��������1��q��}^{;����o�����"4�����i�Q@���f7�7�����N{H\�f����������L�Y��g�o��a�_[ ����C|�
���
0t�������J�?��2m�w�e"��B4������Ou1s?�w�C�y.�����k�^��w����H:}��KX��>j��<��0f'���Q�-�� 6R!I�!��+���X_eRx{�U5)�}!������
� o�V�/��/$�a4?]���d������
���=�?;��{j����l�����54�[Z(���X���m�V��f��MDu3E�q�F� dL��|�����C��F �s���@�3H�!:��������o��6�{��V�:�tOV�'�{��0���p���k��b[��o��6q?����L{*S���G���R`E��Q��v&,FeH�"����"dC�2�������bD��������-,i�%��Jt���+���,ARw�0�����U!�H�IGW��XC�2)tje��a���Ry��3�9*oaN�����U��X�B[+W��+DY��c�Xt*d����Ty)!��������"*T�H�J�S'�Gc��%x��#i"?V������^��F@i�(/�K4�0���f��%hY���� �MeL�`��1�L5�K"�V��1&cD�8g6j�3�G�����,�NK@ec�P���0������a&&V��#�`#W9LfxJ#�\��)����H�)�,d�Jud56�L&�-�<��+j� E)c���� �Z,U��X����o��e`���<���+J7+�1�N
U��(����N���2��n�P Q��P���������� �.9��R��[�M���M������j!�$�������G �lC\�!��l�O��L���M�=I���$��I���/=_,*4#�����$�q�Q9=�t��Uy��!����\���e����!���o�5AC��Q�$NC �A.�/�n)Z�$�i����(q�3�fR���C,q�V��E�8.�HfT�*c5"�FJ���5Y�Z�N�2J� �.6=��aJecYy�%��|�����)��Cb��q�f���4F?�`S�!�*
� �2-��:$!6Q�� ��YrC�6���C��h���I�r���� MXQPM@)�`T��Yc�lZ��V�'Z6�-C�P�B �}����R���K��V�
ZW�C����Lf�&�,�?H$
s���s�WO���8L6�sK�2�j.:�J���+�M0Y�*���v]�#1O��d�\���RIT�o�pg[�L����*��P�n�m 5�$#h��n��X�Efl�f\+i1�d�+�:Z�w�5���1�Np^c��������d�Oy���:��MzC�c*���� {M$B�%��r&�^5=��wr��i�8�F lN�#�@�?������7=G+�3{��=������=�&{G ,8�#�K~2=��a����C���`�aY7*���r3��3f�*�Sh��{�
,�,�F�����-$NE�/S�:����������C,�!�VV���8Fz,�z���5Ac�VDP!����e��M%C����������Z���|����5���#v��;������8f�&h�}l��f�nZ�m��&��z��/���)�T��i��X��=F6T���!WSl&��B�`��3k/���O��X������0�Np�"@@S�n����Y
��h���;�d�B��,Q���F~�s�������8s��[b[a���;������/e7����^9�%]��wy�
�7yM?p@[�CgR�H�I�hi�mU�9��OB�v��J��^����x4��I�d��h�:�2��\�������
I�#��,XS�������:���r&��E��AK
����G��&��L{]��p����"'��T��i����.����;�!~5a��0�i<����J��,��,���IO��� F�q�G |��@�����^�az
Ni�\y��#a���'��@r�[�����1;��Q�]��
AK�`Q�����24��e0����5&�P�c��X��/���qX�
����5AC�ak<l�����%��D8�������qdI������5��P�c��X��+g�c����t+����gM����0 K*�����Z���>!�Q���t$�rY�1���,v��D!�cE�j���&��u�%�n&�,f4�m>�����A�G��,"TO��n�&�D�1�6<Cm��<F37���1�&�i�!.�iVt�p�A3�oZI�]��C��R46�p6f��8}�,��
�H8�%u�����N����z�D���G|l�j2IaR�k-�\
W��
��1�w�*�� �w����\R������~"Y8$`l�-
\�BG�c����8-3d��
<-v�2t�O���{�t�U���^E�v���.��P��kc��Pv�kBy��f=a>�x���'����<���'����P<���kBy�����P��'\c�N_��*������+���8��5�N)�v��� Z�1�c8.�[��P.����������;��w����S�1A9�f^��|�r��C��Gq�� s�����c �
_C�2��)��O6�1L��W�H���e�a�eCYy�����r�]��q!5��& ,p" ��g(hZ��Z�yV����q�8BV�N��nz��G�-�����cW��pJ�{J3bM�)^g�^_��
[C�8�O1��hq^g;c[��f}�'�����x�&R'�sm��W���<F�8�����(�j,�|O�������d
�|��`���Y��H�������������X���5 �n��Y�M0�X2���@v����P}%���������z�y����N�
��y��niph��D����1mw�@�/8r������@v������?�N�x������f������.4.�����T�����5 qW.<��1���}Ys�=���qO����~g�<����%�,_>B�@���
%�h�
%����eM8�����g���
%�h �:,W�%���lqcah�1U�GlsK*�T�%�F^�FK�{�l�<��(��#�5��u��~SmX�������aI��.���1)����;,�PR���P�-������k?�x��c���u}|���fox�[�_������m�����)�����������
�-��od���s�]�g��� �� PK ! ���}m 1@ xl/worksheets/sheet3.xml�[�n�8}_`����(�E�d$=�,;��X����8��8k��g�~/"U%9����N����a����n���y�}{8���ws�����u���~����_�O�|v<�_���������q���������������f��z��?�No�����i��>f���+,�������������������ss���������#>�������o��l_O��a��>���O��c��e�w/���oo?m�/op�y��;����g/��/_^����g��7����������ns�����n����\��7����a�X�g�����^.����|�u�g��q�=;����}<-��� �r>�#�y��j��<��s8=n����l���[�E*���\;�o4r[�����Q��a��}\{>�s��o����BDg<X:�������3�v���;{��x����������n.���Y�g�o������ �]���C|�
��68z����k���Z�?n�r��w��� n������u���������TZM������]���n�3��t|[�9(�696��B�-�n����0�?����;�` �"M���������6llRV������h$�a\>���1����1u�4���pc���dl����|������g��G�6�����?������'�LhXp�(���`��r���i�����{$���w95����O8�'�s����G\�5�y!,��9>L' )d!6u���
�� ,/��ua{���8�6��W��+m�2�4������*�k9��r�pj\�Q��n>�)'��i\+b3h����{(�����2��������u�J'K�����}�\�L��s��-k)����u�b�K/�ph]PY�oD���X�g�*�2�e9Ly�
HK���uV����_��27�|w�AYJ��C��r�����9d/�9�a^�Y�'V�
zg]�`�TUOa>�p�VJ���l�t�}�J��"�3�9��������| ��#�I����e0{
Dn��Lc�]�Zee-c12V���d����|w]AT�E������x-��#��[#{�%Qf���`$*�MV��j|�����*��-�C���K�]�\�'6�������S�HwB�c2�� +�t��&`�+
E�:���R���N�N�{K�u.�L�)��mE�X~�U�D�)���h!�G(��$2�w ��@V����{�,���e0ke���n=D�o����2�����".eU�"�kp�?����e��-01`��R��D������8��������]���^H_��5�-���������!$1�r�e4�=���]��+�>[���K����f���@3����
�(����j�� �oi�%U|�Qu}�u.��ee���U��Y���UUZ�\��UQgR�i5P���A�Z�L�)���A���"���'�@�P��/��/�i���k"�3�ba��9�"�J�C��u�D6(0��A�Z��`3�c�Zg�L�'�)q�W[5���1M�x�X���������.����*GQ��"�
���V;�QP>J��Jh���(��"��Q'<
G�%;�T,���QW�&�x��4�n��Cp���W�L%2(����lG����ej�Rw��Q_m�uA��`���K�Xs�5Q@�����D�m������ ��14n�u)�|-�h�C�)&4%���2e]0���7sNs �(0Iq��/�N �5r�pEO�+��cW��X$6w�w-DI�:A)c����V��,�6sN�s`���n��i��l�-�JH����h]c8J}�Q46��3�:�����
����Y�/�5I���+�s���q��-�/� H
B��$�� ���wx����D;d�8\����6\
W^g' )��i;9'��V=��L�6���hx�?g+��d�gl��mp��Z#��*�����}��,3e��u mq�"_v R���r}���"�Rq!�C�`����I��HK��,�������oE�J�5������C+$`�$~bN�^_o����s��D���Q�f�!�Nb��5[�f�s��F%��Oh������2��������UV��)q�W]=Qu�8���X�g:WW�Q����]����t"��0*�=�����D�����Rw�Q/�� E��&b,ucq��:����N�/N�t�'`6����h��Z"��2ME����h��*��R@S���u�3����,�4sN(s ���)*���@�Tx� Rl8�W��*u�i~f�c ��2���v�b�7�?��5sNs�ze�2P�8?����S�b$��"ha�)����*�*�e_��aNj8�4�h��eA��pak�eN�t�\��������r��K;a������W(}�����fp�2P��+m����C�&�j������)����������^�D�W!i8�2��4,����|�|��-A����5��i#k<�D��A�1�4��'�L��
Z�+(�4M�x���/���UA��~�;O#��h Nt���w��+�FE�Y.�jc���h����������\��K����k���Q�<���O�M:��-A�
u���g�Ac�W�iwzp���chl��4�i�]_Y�����o�X���7�s����,���
M<v(���(����WE�����(���xR=X
'v u������94�F��c�+�`�� ��4#��
k�$,�A�01��J��sG�v+{�1�S���]�)f���Fg%���
��b
�������j����f�$w-�2*��D�s�Q� #>R����@���E�]M�������*SgEz�B�����J ���� p`�����\�����i��$�J)cJ�]�k ��h���/�4I�����s���
����d4�^P���YE/Q��P���������
W�wt�
W�����$M{�����W��n.4�e�*�`�S��9�Z�
%������A�����4%Nk<�M������v����;I� ���e��� ��Z!�C.���s���X>6����u�gTu�
&�A�`���;���'D{snIQ^_^��a0��9d�,�9��=D�lQ�8$h]AtH=b��|�;������/�#�C��?)����u������*����p�X��z8�C�UP�X6�w>t�����b����KWM�3(�1D���Y�d]�#�+����a6R�c�+�`�dYl:�(���d8`S�=��s���0�d[�t�1rE���R�S���+�F]��0E=�� .u��1�r�����������'3P`l���������+
�.�TblR����@ ��"9ei�esN �������z'�z��;� ��S�3��b.+�W\�c6��_�����l�f����s�;�/����q��������O��rx{���{�9fi`i&-KX����rp��^��=���-
,��'�I,,:�?{�/��{�cKK3i���b9i�V��������(�xEml�[|��������E�������7���jq�3�cKK3iY�������c��v��]���l]��^��g�,n_��J����v�5rwQ�y�������j��bk>{��O�l#�e�O� �� PK ! f��q @ xl/worksheets/sheet1.xml�[�n�8}_`�����"E�b$=�,��.������q:F'q�vw�����E��$�3���:*J���C������������?���E��g����~���v���������y��<^v���w������������nw�����v�x>�.���q��9%���,�������,N����������i�/�7�����<�����a��5�������99��6g<��q�z��=o���ys��������.>���������y�����p�|~�{�&�f������o���������=����E���7�{���}v�=�����-�|������������y����i�=���O��������y�{z��"��L�|>��[?������4�����w�QJ��m���htZ���{������������������<��(*Q�����������4�d�����s��7�Z7��������sY&y��\��l��t><���������7�o���*/��[7�{���;�R��M�����;�)�������ww[��W������Q$EQd*��#�v\�9o>�?fE�s_7fL�%���
:�@��v^ �n�'����Y|Gl=�Bd ���n���L�?�T
1�HC���1`cc"�s���Y|�z�����1���n�F��?������Fj�_����q&?��
8x�P�����P���~���f,�m��#E�~�N�d]���#��V{�Q��\�@b��0��`��u���k�w��BP�������]k�uj�Owe5���+�.�=�_�9�CbX��B.�$�����k^������D�I�yj�����&��u�K���"){�m}4��i�UIV�������2�@��*�(�%e��DdE �4O���k�2.(_�Ej�A_,V���JQ�-�
���IQ���b8x�\��e��N2U���
����D���#%���k3.(a�w|�uG�s%��&T�������9^Jg)��5Ag���(�����F/��� _S�����| �t�j��=���=a�(�$��
�B���A���@�,�D�*P��
�E�gq�^"�l*a�#��w-;�!c�&��7{�T�e�&����J���2��\� Z���L�. ���]�L�q�_��i�+��+1�X��=&��]'<�y����+��#���2��L1�I�2�T|Q�N�;�2-2�9��[�[T:��+A����5$�D��j�0��"a�����(Q+�XCA%�H���2P�)l����)s�����.c�S������qw��/y�^�@�u���%]���y�p�-����!�����u�t*��_�b
To^tM���S��]������_A��XRX�@b�iG �����U��Q��� q"�e��7�49�5C7�J�zRE5��bM�P��[e��
�]Z�F])�~a�S��������Y���q��b������3LC0ZT�������G:0�xn s��iK���������Y��^������f����>��k����(.E` e����L���t> ���j`!~:��LM����:\8`wX/
qo����=w��k�]������(DI195B\fE����,��eUb���,�]M�������f����w0����f���6$c�!���;*kqM������b��a���e�2��8�(w���2��q��D�1���7{�J�J��hl6S6D�$����+�T����RP�$K%�)�+�C�����M�N�=����fG>=�4���PP�5|5�jd���������)(����������t4����7Upp�H" �Q]�n�
V���u��Z���(��
4��;a���@X��$���J�)Q���rU������
\i���T����vr)N��U��]��a���H�[y��nX�@f�E��9�Ve�W^V�
-W�3����I�X3"_h�L/�jXTc��u�{����1WA��r04��)R��-���Z��JU"�����,��+E\�]G��^S�����<u�Q��]y�:�����z��hC��B?h:�{M���9VA���n):E��e6��r8����r��(��8����I4��h���vDC@��e�����>��E����2���O����u�N[L�����6��� S�&%Y/����t�5fX
?�^4\c�����!�K���S
���a��t!���T�1�$�7{����yvoW�5���"C>LYCk�Ia�<Q�
J�.�#���N�b2�
��J!�ub�1�$�7;��LT��MC�*�0wg
�)FT��<8����\B/��aL3����5�>�U_/h����bN������C�[�l<�F �I�#��o�
����Y�����JO��4��[�w���$V#���������s� q$�
����OWi���z�J������"\tf�D��m Fb�C>(��Cuz��n/F?�1�T�#�����;�
�A�coHU������7����,3�&tX5��a{�%��TI�BMcmz���4T �:����=u��I���!�����c!�����r!eDD��_Q!���~>S��+�u�DW�c��\o��a1��okh���N��wM ���e'�����e�O�j�3H�����t*��`�����6T��� S
�D�V�l�4�+(�RFQ��X�X��D#j�|-A��B����0��/wY�;V�k���s��sW"�$��
E���3�� Lr+�w�&u�Rt�b���H������77.c,E�sI�z�cL��&u����!h���9V?Hk����2y'��wK�y)�$Ua9$�)�Lr���s�����zm5p>��[ok`yK�@��jF ,��4p�
����:w�������\i���4��k��
�O����LT��t a+o��.U�@�f���*K3��"���
,�,~d��24>��E���(�WW��G�-�u�8�b����7w�)I��I�yC����
�cZ�I:���o�������k Z�l����bR_�r8���=#\��0>���y���|���q�XP�}8��B������|M��%��^V��ust5F4�nz�-F�-W�c��ao��a������t�A��P���5AC+,�]��@��J`x��)w��u��0.������1�
��7{��FkfXl�7�u���� ?d�S9�8��M�1{YHK�;<�*��6}
�0.a��j������V�%�
6+��d|?���s
�EZ^�q����[J�zCcB�
5l�-r�bk�7k������=0+v�e���`�|��S�-J��� ��>P��g������"�pH�#��u'0�w�/���i�=|3a����M����j������������.W���k�Cowy����ZjX�Q�
����t�Bg���ry���CK
K=jY���`��l�z�Sby�� CK
K=jY���`��������;��w8�6��,��sZp�oi��
-
,� ,���8��������e�r�=�(�94�i�������!o{���p�����8H��?s�z�p8����������u��y�?��������G��I�����x>n�g��������-�p���� �� PK ! !��� � xl/sharedStrings.xml��Mo�0���a@�
�-'��v�Xw�-��Pm��W��f�~J��� ��|��+�������r�JB���r���a�3� X�T��VX�Zr]}�TX���U�$�s��8�M���+=���F���4]l���=��"N�dK��F���$�2���#.�#3J����p�����]U���{������v�����_Ch��`�.�T��l�u���aB�8
�����;f��c��M �����/J�Hzf>��O���8�8��%�q��|���A ��,���
�`�e�,�� �����`�3X���Zmx7�����Y��`[?����0���w�!$�k�e�~�&�lJ��k������G�O�I>)���Z����<k����� �1m�}��>�/lz��e��hy����`(M�H�i�4K������{%b���� �� PK ! �t9 w
xl/styles.xml�V[o�0~�����&P�JR��H��jR��W�8��/����i�}�vT��n��������_�8���T���"����*A��Yo�m�(0��$hM4�J?~��Y3��$��B�-��'a��%�X���8)��� ��P���B[!��A]�S��� �OQ��zl�^.y�
]PF���B�'���
/@]��8�t;��zNs%�,��eY���E9�!hJ�R
��\6�$h�������"�G��+��K����Q���dR"�����x�k��BQ�VbN��o��f��)�f7C���I����G���Q�"AY��Ew�c�+
��ct�W�m��RC,)c��la#���Q""h��u
)���;�])��.vBg�'U�����������@8���kd
�iTjWR`�����V��3A�����W n�l0�l����f �a=��;����9HQN{�N}/_�kU��7�E�`|������v�c� ��!��
w!x@;XF�Cj-��j�Uy���>L '
\�lm����������gF+��gIc�:��R���*�s����<���<+\���yh������y�t��i[����
���/���e�-�m����'id�Z�u1�T���V��F��/{�$��������[4�*6��_ �hv
�Yu����� 7���O��
�
R�����0A��7��H���>I�T$h����im@��i���
E��f�i<���Q4�����7���z���l���At���/� <O����^f0���f�.R��-���^�v�]u r�9���1� �� PK ! j�C� � docProps/app.xml �(� ��Mo!����������,�(u���KvR)�j��zi0 `�v~}�]9^�>D�
3//���/w[M�AY���8����2����oF( L����o����}TH�0!�U�n�X�n!�S��Ji�bZ�
�e�$^[Yo�D6���wM������q���5-�l���z���W�i%!�.�W%�
����N��lX��n���*�E��p�W4.��(A�������-A� x�
�h= �9]���G�����������Ek�?�
b�%A����v�1�)8�=H*�"�U���K���lH�1��=N��c���.�4;�����w�X�<j�����{^����4k�9�����[���7�/CM��t���o��< �4��d��2O����5D<<�i�����l���Mo�uk���l�8h�-�cy��=1�g�,M� �����? �� PK ! yyS`� � '