parallel mode and parallel contexts
Attached is a patch that adds two new concepts: parallel mode, and
parallel contexts. The idea of this code is to provide a framework
for specific parallel things that you want to do, such as parallel
sequential scan or parallel sort. When you're in parallel mode,
certain operations - like DDL, and anything that would update the
command counter - are prohibited. But you gain the ability to create
a parallel context, which in turn can be used to fire up parallel
workers. And if you do that, then your snapshot, combo CID hash, and
GUC values will be copied to the worker, which is handy.
This patch is very much half-baked. Among the things that aren't right yet:
- There's no handling of heavyweight locking, so I'm quite sure it'll
be possible to cause undetected deadlocks if you work at it. There
are some existing threads on this topic and perhaps we can incorporate
one of those concepts into this patch, but this version does not.
- There's no provision for copying the parent's XID and sub-XIDs, if
any, to the background workers, which means that if you use this and
your transaction has written data, you will get wrong answers, because
TransactionIdIsCurrentTransactionId() will do the wrong thing.
- There's no really deep integration with the transaction system yet.
Previous discussions seem to point toward the need to do various types
of coordinated cleanup when the parallel phase is done, or when an
error happens. In particular, you probably don't want the abort
record to get written while there are still possibly backends that are
part of that transaction doing work; and you certainly don't want
files created by the current transaction to get removed while some
other backend is still writing them. The right way to work all of
this out needs some deep thought; agreeing on what the design should
be is probably harder than implement it.
Despite the above, I think this does a fairly good job laying out how
I believe parallelism can be made to work in PostgreSQL: copy a bunch
of state from the user backend to the parallel workers, compute for a
while, and then shut everything down. Meanwhile, while parallelism is
running, forbid changes to state that's already been synchronized, so
that things don't get out of step. I think the patch it shows how the
act of synchronizing state from the master to the workers can be made
quite modular and painless, even though it doesn't synchronize
everything relevant. I'd really appreciate any design thoughts anyone
may have on how to fix the problems mentioned above, how to fix any
other problems you foresee, or even just a list of reasons why you
think this will blow up.
What I think is that we're really pretty close to do real parallelism,
and that this is probably the last major piece of infrastructure that
we need in order to support parallel execution in a reasonable way.
That's a pretty bold statement, but I believe it to be true: despite
the limitations of the current version of this patch, I think we're
very close to being able to sit down and code up a parallel algorithm
in PostgreSQL and have that not be all that hard. Once we get the
first one, I expect a whole bunch more to come together far more
quickly than the first one did.
I would be remiss if I failed to mention that this patch includes work
by my colleagues Amit Kapila, Rushabh Lathia, and Jeevan Chalke, as
well as my former colleague Noah Misch; and that it would not have
been possible without the patient support of EnterpriseDB management.
Thanks,
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v0.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v0.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..2a7251c
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,7 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_dummy(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
diff --git a/contrib/parallel_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..99b830f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,82 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/parallel.h"
+#include "access/xact.h"
+#include "fmgr.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_dummy);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct {
+ int32 sleep_time;
+} ParallelDummyInfo;
+
+void _PG_init(void);
+void worker_main(shm_toc *toc);
+
+Datum
+parallel_dummy(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelDummyInfo *info;
+
+ if (nworkers < 1)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be positive")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy", "worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelDummyInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelDummyInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+void
+worker_main(shm_toc *toc)
+{
+ ParallelDummyInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index df4853b..31efa37 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2636,6 +2636,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3077,6 +3087,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..e427275
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,36 @@
+Overview
+========
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). These functions
+don't launch any workers or directly enable parallelism, but they put in place
+a variety of prohibitions required to make parallelism safe.
+
+To actually parallelize a particular operation, use a ParallelContext. This
+establishes a dynamic shared memory segment and registers dynamic background
+workers which will attach to that segment. We arrange to synchronize various
+pieces of state - such as, most simply, the database and user OIDs - from the
+backend that is initiating parallelism to all of the background workers
+launched via a ParallelContext. The basic coding pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..eeae591
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,787 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0007)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(StringInfo msg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers > 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ shm_toc_estimate_keys(&pcxt->estimator, 5);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ * We do, however, give up on registering any more workers once
+ * registration fails the first time; no sense beating our head against
+ * a brick wall.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ if (!RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ break;
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Destroy a parallel context.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /*
+ * If any background workers have been started, terminate them all.
+ * To avoid leaking memory, release the background worker and the message
+ * queue handle. This doesn't actually detach the message queue; we'll
+ * take care of that by detaching the dynamic shared memory segment itself.
+ */
+ if (pcxt->worker != NULL)
+ {
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ {
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].bgwhandle);
+ }
+ pfree(pcxt->worker[i].error_mqh);
+ }
+ }
+
+ /*
+ * 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Free memory. */
+ pfree(pcxt->worker);
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler) && InterruptHoldoffCount == 0
+ && CritSectionCount == 0)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh == NULL)
+ continue;
+
+ for (;;)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(&msg);
+ pfree(msg.data);
+ }
+ if (res == SHM_MQ_DETACHED)
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker is insufficient justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * XXX. We should probably use the error context callbacks in
+ * effect at the time the parallel context was created.
+ */
+
+ ThrowErrorData(&edata);
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *combocidspace;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /*
+ * XXX. At this point, we should restore the transaction state as it
+ * exists in the master. But since we don't have code for that yet, just
+ * start a new transaction.
+ */
+ StartTransactionCommand();
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * Note that it's unsafe for this entrypoint to be a function living in a
+ * dynamically loaded module, because it might not be loaded at the same
+ * address in every process. Use CreateParallelContextForExtension()
+ * rather than CreateParallelContext() to handle that case.
+ */
+ fps->entrypoint(toc);
+
+ /*
+ * XXX. There's probably some end-of-parallel-phase cleanup that needs to
+ * happen here; we shouldn't just exit in the midde of a transaction.
+ * But I don't know what's needed yet.
+ */
+}
+
+/*
+ * In-core trampoline to invoke extension entrypints.
+ *
+ * See comments in ParallelMain() and StartBackgroundWorker() for why we
+ * need this.
+ */
+static void
+ParallelExtensionTrampoline(shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(toc);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index c541156..92f657e 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8b2f714..2fa2c11 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -152,6 +153,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +184,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -642,7 +645,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -789,6 +801,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +861,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1772,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(!IsInParallelMode());
+
/*
* check the current transaction state
*/
@@ -1835,6 +1904,8 @@ CommitTransaction(void)
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("CommitTransaction");
/*
@@ -1921,6 +1992,13 @@ CommitTransaction(void)
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOXact_Parallel(true);
+ s->parallelMode = false;
+ }
+
/*
* Let others know about no transaction in progress by me. Note that this
* must be done _before_ releasing locks we hold and _after_
@@ -2040,6 +2118,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2391,6 +2471,13 @@ AbortTransaction(void)
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOXact_Parallel(false);
+ s->parallelMode = false;
+ }
+
/*
* Let others know about no transaction in progress by me. Note that this
* must be done _before_ releasing locks we hold and _after_
@@ -3540,6 +3627,16 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3594,6 +3691,16 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3694,6 +3801,16 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3806,6 +3923,16 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3860,6 +3987,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4021,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -4157,6 +4304,13 @@ CommitSubTransaction(void)
elog(WARNING, "CommitSubTransaction while in %s state",
TransStateAsString(s->state));
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Pre-commit processing goes here */
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
@@ -4315,6 +4469,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4616,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index 4a542e6..04ecbbb 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -476,7 +476,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 08abe14..4790df5 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index ba5b938..0b422dc 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 8c799d3..9223f5e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 4d11260..62b615a 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index cfa4a24..72d55c7 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 6e6b429..b07978c 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 1d6e3f3..8fb39d3 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -471,7 +471,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1665,7 +1665,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..451f814 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5305,7 +5305,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d953545..7b7b57a 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index cd9a287..0678678 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index f126118..dd53426 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1656,6 +1656,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..4285748 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -37,6 +37,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3704,7 +3706,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index aa8fe88..c4862a9 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 8fccb4c..a045062 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -350,11 +350,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -365,7 +364,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index c348034..8f6d517 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b1bff7f..a30d809 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index a70c754..d2f23c0 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -287,3 +288,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids || !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index d601efe..b823827 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,13 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot update SecondarySnapshpt during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +372,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +420,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +584,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1296,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1399,152 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ *
+ * If set_transaction_snapshot is true, the snapshot is additionally installed
+ * as the transaction snapshot.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Size subxipoff;
+ Snapshot snapshot;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = subxipoff = sizeof(SnapshotData) +
+ serialized_snapshot->xcnt * sizeof(TransactionId);
+ size += serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ memcpy(snapshot->xip, (TransactionId *) (serialized_snapshot + 1),
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ memcpy(snapshot->subxip,
+ (TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..9fe2df8
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,58 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+
+typedef void (*parallel_worker_main_type)(shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index b018aa4..6e8290f 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -260,4 +260,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index f8e68c9..78583b3 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 1558a75..fb7754f 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -258,6 +258,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -290,7 +291,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -391,7 +392,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a3b3d5f..273cecb 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -130,6 +130,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 0c4611b..9f4e4b9 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index c625562..3046e52 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index 9e482ff..232f729 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index abe7016..7efd427 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On 12 December 2014 at 22:52, Robert Haas <robertmhaas@gmail.com> wrote:
I would be remiss if I failed to mention that this patch includes work
by my colleagues Amit Kapila, Rushabh Lathia, and Jeevan Chalke, as
well as my former colleague Noah Misch; and that it would not have
been possible without the patient support of EnterpriseDB management.
Noted and thanks to all.
I will make this my priority for review, but regrettably not until New
Year, about 2.5-3 weeks away.
--
Simon Riggs 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, Dec 17, 2014 at 9:16 AM, Simon Riggs <simon@2ndquadrant.com> wrote:
On 12 December 2014 at 22:52, Robert Haas <robertmhaas@gmail.com> wrote:
I would be remiss if I failed to mention that this patch includes work
by my colleagues Amit Kapila, Rushabh Lathia, and Jeevan Chalke, as
well as my former colleague Noah Misch; and that it would not have
been possible without the patient support of EnterpriseDB management.Noted and thanks to all.
I will make this my priority for review, but regrettably not until New
Year, about 2.5-3 weeks away.
OK!
In the meantime, I had a good chat with Heikki on IM yesterday which
gave me some new ideas on how to fix up the transaction handling in
here, which I am working on implementing. So hopefully I will have
that by then.
I am also hoping Amit will be adapting his parallel seq-scan patch to
this framework.
--
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, Dec 17, 2014 at 2:53 PM, Robert Haas <robertmhaas@gmail.com> wrote:
In the meantime, I had a good chat with Heikki on IM yesterday which
gave me some new ideas on how to fix up the transaction handling in
here, which I am working on implementing. So hopefully I will have
that by then.
And here is a new version. This version has some real integration
with the transaction system, along the lines of what Heikki suggested
to me, so hopefully tuple visibility calculations in a parallel worker
will now produce the right answers, though I need to integrate this
with code to actually do something-or-other in parallel in order to
really test that. There are still some problems with parallel worker
shutdown. As hard as I tried to fight it, I'm gradually resigning
myself to the fact that we're probably going to have to set things up
so that the worker waits for all of its children to die during abort
processing (and during commit processing, but that's not bothersome).
Otherwise, to take just one example, chaos potentially ensues if you
run a parallel query in a subtransaction and then roll back to a
savepoint. But this version just kills the workers and doesn't
actually wait for them to die; I'll see about fixing that, but wanted
to send this around for comments first.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v0.1.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v0.1.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..2a7251c
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,7 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_dummy(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
diff --git a/contrib/parallel_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..99b830f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,82 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/parallel.h"
+#include "access/xact.h"
+#include "fmgr.h"
+#include "miscadmin.h"
+#include "utils/builtins.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_dummy);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct {
+ int32 sleep_time;
+} ParallelDummyInfo;
+
+void _PG_init(void);
+void worker_main(shm_toc *toc);
+
+Datum
+parallel_dummy(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelDummyInfo *info;
+
+ if (nworkers < 1)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be positive")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy", "worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelDummyInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelDummyInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+void
+worker_main(shm_toc *toc)
+{
+ ParallelDummyInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index df4853b..31efa37 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2636,6 +2636,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3077,6 +3087,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..e427275
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,36 @@
+Overview
+========
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). These functions
+don't launch any workers or directly enable parallelism, but they put in place
+a variety of prohibitions required to make parallelism safe.
+
+To actually parallelize a particular operation, use a ParallelContext. This
+establishes a dynamic shared memory segment and registers dynamic background
+workers which will attach to that segment. We arrange to synchronize various
+pieces of state - such as, most simply, the database and user OIDs - from the
+backend that is initiating parallelism to all of the background workers
+launched via a ParallelContext. The basic coding pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..5098e0a
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,792 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(StringInfo msg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers > 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 6);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ * We do, however, give up on registering any more workers once
+ * registration fails the first time; no sense beating our head against
+ * a brick wall.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ if (!RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ break;
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Destroy a parallel context.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /*
+ * If any background workers have been started, terminate them all.
+ * To avoid leaking memory, release the background worker and the message
+ * queue handle. This doesn't actually detach the message queue; we'll
+ * take care of that by detaching the dynamic shared memory segment itself.
+ */
+ if (pcxt->worker != NULL)
+ {
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ {
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].bgwhandle);
+ }
+ pfree(pcxt->worker[i].error_mqh);
+ }
+ }
+
+ /*
+ * 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Free memory. */
+ pfree(pcxt->worker);
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler) && InterruptHoldoffCount == 0
+ && CritSectionCount == 0)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh == NULL)
+ continue;
+
+ for (;;)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(&msg);
+ pfree(msg.data);
+ }
+ if (res == SHM_MQ_DETACHED)
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker is insufficient justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * XXX. We should probably use the error context callbacks in
+ * effect at the time the parallel context was created.
+ */
+
+ ThrowErrorData(&edata);
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * Note that it's unsafe for this entrypoint to be a function living in a
+ * dynamically loaded module, because it might not be loaded at the same
+ * address in every process. Use CreateParallelContextForExtension()
+ * rather than CreateParallelContext() to handle that case.
+ */
+ fps->entrypoint(toc);
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+}
+
+/*
+ * In-core trampoline to invoke extension entrypints.
+ *
+ * See comments in ParallelMain() and StartBackgroundWorker() for why we
+ * need this.
+ */
+static void
+ParallelExtensionTrampoline(shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(toc);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index c541156..92f657e 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8b2f714..a55895e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,21 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * This stores XID of the toplevel transaction to which we belong.
+ * Normally, it's the same as TopTransactionState.transactionId, but in
+ * a parallel worker it may not be.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+
+/*
+ * If we're a parallel worker, there may be additional XIDs that we need
+ * to regard as part of our transaction even though they don't appear
+ * in the transaction state stack.
+ */
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +128,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +170,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +201,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +371,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +386,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +480,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +538,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +671,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +774,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +857,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +917,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1828,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1959,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1845,6 +1973,10 @@ CommitTransaction(void)
TransStateAsString(s->state));
Assert(s->parent == NULL);
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/*
* Do pre-commit processing that involves calling user-defined code, such
* as triggers. Since closing cursors could queue trigger actions,
@@ -1867,7 +1999,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1915,9 +2048,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2081,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2130,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2156,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2182,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2428,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2449,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2498,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2384,10 +2533,22 @@ AbortTransaction(void)
AtAbort_Twophase();
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2566,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2590,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2479,6 +2644,9 @@ CleanupTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with abort processing, set current transaction state back to
* default
@@ -2531,6 +2699,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2735,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3023,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3413,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3591,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3694,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3733,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3765,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3800,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3835,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3913,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3949,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4038,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4074,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4107,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4141,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4160,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4236,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4328,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4157,6 +4427,13 @@ CommitSubTransaction(void)
elog(WARNING, "CommitSubTransaction while in %s state",
TransStateAsString(s->state));
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Pre-commit processing goes here */
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
@@ -4315,6 +4592,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4739,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4787,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4984,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index ed2b05a..5398b70 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -476,7 +476,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 08abe14..4790df5 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 811e1d4..d9b0aed 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 8c799d3..9223f5e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 4d11260..62b615a 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index cfa4a24..72d55c7 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 6e6b429..b07978c 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 675f985..637749a 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -471,7 +471,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1664,7 +1664,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..451f814 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5305,7 +5305,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d953545..7b7b57a 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index cd9a287..0678678 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index e783955..b35cbbd 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..4285748 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -37,6 +37,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3704,7 +3706,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index aa8fe88..c4862a9 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 8fccb4c..a045062 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -350,11 +350,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -365,7 +364,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index c348034..8f6d517 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b1bff7f..a30d809 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index ea7a905..33b0b52 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids || !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index d601efe..b823827 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,13 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot update SecondarySnapshpt during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +372,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +420,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +584,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1296,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1399,152 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ *
+ * If set_transaction_snapshot is true, the snapshot is additionally installed
+ * as the transaction snapshot.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Size subxipoff;
+ Snapshot snapshot;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = subxipoff = sizeof(SnapshotData) +
+ serialized_snapshot->xcnt * sizeof(TransactionId);
+ size += serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ memcpy(snapshot->xip, (TransactionId *) (serialized_snapshot + 1),
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ memcpy(snapshot->subxip,
+ (TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..9fe2df8
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,58 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+
+typedef void (*parallel_worker_main_type)(shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index b018aa4..91d9d73 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index f8e68c9..78583b3 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 1558a75..fb7754f 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -258,6 +258,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -290,7 +291,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -391,7 +392,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a3b3d5f..273cecb 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -130,6 +130,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 0c4611b..9f4e4b9 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index c625562..3046e52 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index 9e482ff..232f729 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index abe7016..7efd427 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On Thu, Dec 18, 2014 at 4:53 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Dec 17, 2014 at 9:16 AM, Simon Riggs <simon@2ndquadrant.com> wrote:
On 12 December 2014 at 22:52, Robert Haas <robertmhaas@gmail.com> wrote:
I would be remiss if I failed to mention that this patch includes work
by my colleagues Amit Kapila, Rushabh Lathia, and Jeevan Chalke, as
well as my former colleague Noah Misch; and that it would not have
been possible without the patient support of EnterpriseDB management.Noted and thanks to all.
I will make this my priority for review, but regrettably not until New
Year, about 2.5-3 weeks away.OK!
In the meantime, I had a good chat with Heikki on IM yesterday which
gave me some new ideas on how to fix up the transaction handling in
here, which I am working on implementing. So hopefully I will have
that by then.I am also hoping Amit will be adapting his parallel seq-scan patch to
this framework.
Simon, if you are planning to review this patch soon, could you add
your name as a reviewer for this patch in the commit fest app?
Thanks,
--
Michael
--
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 19, 2014 at 8:59 AM, Robert Haas <robertmhaas@gmail.com> wrote:
And here is a new version.
Here is another new version, with lots of bugs fixed. The worker
shutdown sequence is now much more robust, although I think there may
still be a bug or two lurking, and I fixed a bunch of other things
too. There's now a function called parallel_count() in the
parallel_dummy extension contained herein, which does a parallel count
of a relation you choose:
rhaas=# select count(*) from pgbench_accounts;
count
---------
4000000
(1 row)
Time: 396.635 ms
rhaas=# select parallel_count('pgbench_accounts'::regclass, 0);
NOTICE: PID 2429 counted 4000000 tuples
parallel_count
----------------
4000000
(1 row)
Time: 234.445 ms
rhaas=# select parallel_count('pgbench_accounts'::regclass, 4);
NOTICE: PID 2499 counted 583343 tuples
CONTEXT: parallel worker, pid 2499
NOTICE: PID 2500 counted 646478 tuples
CONTEXT: parallel worker, pid 2500
NOTICE: PID 2501 counted 599813 tuples
CONTEXT: parallel worker, pid 2501
NOTICE: PID 2502 counted 611389 tuples
CONTEXT: parallel worker, pid 2502
NOTICE: PID 2429 counted 1558977 tuples
parallel_count
----------------
4000000
(1 row)
Time: 150.004 ms
rhaas=# select parallel_count('pgbench_accounts'::regclass, 8);
NOTICE: PID 2429 counted 1267566 tuples
NOTICE: PID 2504 counted 346236 tuples
CONTEXT: parallel worker, pid 2504
NOTICE: PID 2505 counted 345077 tuples
CONTEXT: parallel worker, pid 2505
NOTICE: PID 2506 counted 355325 tuples
CONTEXT: parallel worker, pid 2506
NOTICE: PID 2507 counted 350872 tuples
CONTEXT: parallel worker, pid 2507
NOTICE: PID 2508 counted 338855 tuples
CONTEXT: parallel worker, pid 2508
NOTICE: PID 2509 counted 336903 tuples
CONTEXT: parallel worker, pid 2509
NOTICE: PID 2511 counted 326716 tuples
CONTEXT: parallel worker, pid 2511
NOTICE: PID 2510 counted 332450 tuples
CONTEXT: parallel worker, pid 2510
parallel_count
----------------
4000000
(1 row)
Time: 166.347 ms
This example table (pgbench_accounts, scale 40, ~537 MB) is small
enough that parallelism doesn't really make sense; you can see from
the notice messages above that the master manages to count a quarter
of the table before the workers get themselves up and running. The
pointer is rather to show how the infrastructure works and that it can
be used to write code to do practically useful tasks in a surprisingly
small number of lines of code; parallel_count is only maybe ~100 lines
on top of the base patch.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v0.2.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v0.2.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..3c0ae7d
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,12 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_sleep(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
+
+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_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..07d2762
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,238 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.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/snapmgr.h"
+#include "utils/tqual.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_sleep);
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct
+{
+ int32 sleep_time;
+} ParallelSleepInfo;
+
+typedef struct
+{
+ int32 relid;
+ slock_t mutex;
+ BlockNumber lastblock;
+ BlockNumber currentblock;
+ int64 ntuples;
+} ParallelCountInfo;
+
+void _PG_init(void);
+void sleep_worker_main(shm_toc *toc);
+void count_worker_main(shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelCountInfo *info);
+
+Datum
+parallel_sleep(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelSleepInfo *info;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy",
+ "sleep_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelSleepInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelSleepInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ WaitForParallelWorkersToFinish(pcxt);
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelCountInfo *info;
+ Relation rel;
+ int64 result;
+
+ 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_dummy",
+ "count_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelCountInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelCountInfo));
+ info->relid = relid;
+ SpinLockInit(&info->mutex);
+ info->lastblock = RelationGetNumberOfBlocks(rel);
+ info->currentblock = 0;
+ info->ntuples = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, info);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ result = info->ntuples;
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(result);
+}
+
+void
+sleep_worker_main(shm_toc *toc)
+{
+ ParallelSleepInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
+
+void
+count_worker_main(shm_toc *toc)
+{
+ ParallelCountInfo *info;
+ Relation rel;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ rel = relation_open(info->relid, AccessShareLock);
+ count_helper(rel, info);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelCountInfo *info)
+{
+ int64 ntuples = 0;
+ int64 mytuples = 0;
+ Oid relid = info->relid;
+ Snapshot snapshot = GetActiveSnapshot();
+
+ for (;;)
+ {
+ 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);
+ lines = PageGetMaxOffsetNumber(page);
+ ntuples = 0;
+
+ all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
+
+ for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
+ lineoff <= lines;
+ lineoff++, lpp++)
+ {
+ HeapTupleData loctup;
+
+ if (!ItemIdIsNormal(lpp))
+ continue;
+ if (all_visible)
+ {
+ ++ntuples;
+ continue;
+ }
+
+ loctup.t_tableOid = relid;
+ loctup.t_data = (HeapTupleHeader) PageGetItem(page, lpp);
+ loctup.t_len = ItemIdGetLength(lpp);
+
+ if (HeapTupleSatisfiesVisibility(&loctup, snapshot, buffer))
+ ++ntuples;
+ }
+
+ UnlockReleaseBuffer(buffer);
+ }
+
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 61216e5..8e3ccf4 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index df4853b..31efa37 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2636,6 +2636,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3077,6 +3087,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..e427275
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,36 @@
+Overview
+========
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). These functions
+don't launch any workers or directly enable parallelism, but they put in place
+a variety of prohibitions required to make parallelism safe.
+
+To actually parallelize a particular operation, use a ParallelContext. This
+establishes a dynamic shared memory segment and registers dynamic background
+workers which will attach to that segment. We arrange to synchronize various
+pieces of state - such as, most simply, the database and user OIDs - from the
+backend that is initiating parallelism to all of the background workers
+launched via a ParallelContext. The basic coding pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..5d9514a
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,884 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 6);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ * We do, however, give up on registering any more workers once
+ * registration fails the first time; no sense beating our head against
+ * a brick wall.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If wait_patiently is true, we wait for the workers associated with this
+ * parallel context to go away on their own; if it's false, we kill them.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill and forget about each worker in turn. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ {
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].bgwhandle);
+ }
+ if (pcxt->worker[i].error_mqh != NULL)
+ pfree(pcxt->worker[i].error_mqh);
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler)
+ && InterruptHoldoffCount == 0 && CritSectionCount == 0
+ && !HandlingParallelMessages)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+ HandlingParallelMessages = true;
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ HandlingParallelMessages = false;
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read messages for as long as we have an error queue; if we
+ * have hit (or hit while reading) ReadyForQuery, this will go to
+ * NULL.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else if (res == SHM_MQ_DETACHED)
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker is insufficient justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * XXX. We should probably use the error context callbacks in
+ * effect at the time the parallel context was created.
+ */
+
+ ThrowErrorData(&edata);
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * Note that it's unsafe for this entrypoint to be a function living in a
+ * dynamically loaded module, because it might not be loaded at the same
+ * address in every process. Use CreateParallelContextForExtension()
+ * rather than CreateParallelContext() to handle that case.
+ */
+ fps->entrypoint(toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * In-core trampoline to invoke extension entrypints.
+ *
+ * See comments in ParallelMain() and StartBackgroundWorker() for why we
+ * need this.
+ */
+static void
+ParallelExtensionTrampoline(shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index c541156..92f657e 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8b2f714..d3174ce 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,21 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * This stores XID of the toplevel transaction to which we belong.
+ * Normally, it's the same as TopTransactionState.transactionId, but in
+ * a parallel worker it may not be.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+
+/*
+ * If we're a parallel worker, there may be additional XIDs that we need
+ * to regard as part of our transaction even though they don't appear
+ * in the transaction state stack.
+ */
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +128,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +170,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +201,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +371,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +386,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +480,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +538,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +671,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +774,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +857,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +917,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1828,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1959,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1845,6 +1973,10 @@ CommitTransaction(void)
TransStateAsString(s->state));
Assert(s->parent == NULL);
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/*
* Do pre-commit processing that involves calling user-defined code, such
* as triggers. Since closing cursors could queue trigger actions,
@@ -1867,7 +1999,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1915,9 +2048,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2081,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2130,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2156,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2182,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2428,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2449,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2498,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2384,10 +2533,22 @@ AbortTransaction(void)
AtAbort_Twophase();
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2566,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2590,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2643,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2700,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2736,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3024,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3414,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3592,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3695,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3734,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3766,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3801,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3836,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3914,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3950,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4039,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4075,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4108,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4142,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4161,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4237,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4329,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4157,6 +4428,13 @@ CommitSubTransaction(void)
elog(WARNING, "CommitSubTransaction while in %s state",
TransStateAsString(s->state));
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Pre-commit processing goes here */
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
@@ -4315,6 +4593,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4740,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4788,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4985,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index ed2b05a..5398b70 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -476,7 +476,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 08abe14..4790df5 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 811e1d4..d9b0aed 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 8c799d3..9223f5e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 4d11260..62b615a 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index cfa4a24..72d55c7 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 6e6b429..b07978c 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 675f985..637749a 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -471,7 +471,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1664,7 +1664,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..451f814 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5305,7 +5305,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d953545..7b7b57a 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index cd9a287..0678678 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index e783955..b35cbbd 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..4285748 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -37,6 +37,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3704,7 +3706,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index aa8fe88..c4862a9 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 8fccb4c..a045062 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -350,11 +350,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -365,7 +364,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index c348034..8f6d517 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index b1bff7f..a30d809 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index ea7a905..33b0b52 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids || !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index d601efe..bc071aa 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,13 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot update SecondarySnapshpt during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +372,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +420,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +584,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1296,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1399,162 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ *
+ * If set_transaction_snapshot is true, the snapshot is additionally installed
+ * as the transaction snapshot.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..b7e4f3a
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,59 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+
+typedef void (*parallel_worker_main_type)(shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index b018aa4..91d9d73 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index f8e68c9..78583b3 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 1558a75..fb7754f 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -258,6 +258,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -290,7 +291,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -391,7 +392,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a3b3d5f..273cecb 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -130,6 +130,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 0c4611b..9f4e4b9 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index c625562..3046e52 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index 9e482ff..232f729 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index abe7016..7efd427 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On Thu, Dec 18, 2014 at 1:23 AM, Robert Haas <robertmhaas@gmail.com> wrote:
In the meantime, I had a good chat with Heikki on IM yesterday which
gave me some new ideas on how to fix up the transaction handling in
here, which I am working on implementing. So hopefully I will have
that by then.I am also hoping Amit will be adapting his parallel seq-scan patch to
this framework.
While working on parallel seq-scan patch to adapt this framework, I
noticed few things and have questions regrading the same.
1.
Currently parallel worker just attaches to error queue, for tuple queue
do you expect it to be done in the same place or in the caller supplied
function, if later then we need segment address as input to that function
to attach queue to the segment(shm_mq_attach()).
Another question, I have in this regard is that if we have redirected
messages to error queue by using pq_redirect_to_shm_mq, then how can
we set tuple queue for the same purpose. Similarly I think more handling
is needed for tuple queue in master backend and the answer to above will
dictate what is the best way to do it.
2.
Currently there is no interface for wait_for_workers_to_become_ready()
in your patch, don't you think it is important that before start of fetching
tuples, we should make sure all workers are started, what if some worker
fails to start?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 2014-12-22 14:14:31 -0500, Robert Haas wrote:
On Fri, Dec 19, 2014 at 8:59 AM, Robert Haas <robertmhaas@gmail.com> wrote:
And here is a new version.
Here is another new version, with lots of bugs fixed.
A couple remarks:
* Shouldn't this provide more infrastructure to deal with the fact that
we might get less parallel workers than we had planned for?
* I wonder if parallel contexts shouldn't be tracked via resowners
* combocid.c should error out in parallel mode, as insurance
* I doubt it's a good idea to allow heap_insert, heap_inplace_update,
index_insert. I'm not convinced that those will be handled correct and
relaxing restrictions is easier than adding them.
* I'd much rather separate HandleParallelMessageInterrupt() in one
variant that just tells the machinery there's a interrupt (called from
signal handlers) and one that actually handles them. We shouldn't even
consider adding any new code that does allocations, errors and such in
signal handlers. That's just a *bad* idea.
* I'm not a fan of the shm_toc stuff... Verbose and hard to read. I'd
much rather place a struct there and be careful about not using
pointers. That also would obliviate the need to reserve some ids.
* Doesn't the restriction in GetSerializableTransactionSnapshotInt()
apply for repeatable read just the same?
* I'm not a fan of relying on GetComboCommandId() to restore things in
the same order as before.
* I'd say go ahead and commit the BgworkerByOid thing
earlier/independently. I've seen need for that a couple times.
* s/entrypints/entrypoints/
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 Fri, Jan 2, 2015 at 9:04 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
While working on parallel seq-scan patch to adapt this framework, I
noticed few things and have questions regrading the same.1.
Currently parallel worker just attaches to error queue, for tuple queue
do you expect it to be done in the same place or in the caller supplied
function, if later then we need segment address as input to that function
to attach queue to the segment(shm_mq_attach()).
Another question, I have in this regard is that if we have redirected
messages to error queue by using pq_redirect_to_shm_mq, then how can
we set tuple queue for the same purpose. Similarly I think more handling
is needed for tuple queue in master backend and the answer to above will
dictate what is the best way to do it.
I've come to the conclusion that it's a bad idea for tuples to be sent
through the same queue as errors. We want errors (or notices, but
especially errors) to be processed promptly, but there may be a
considerable delay in processing tuples. For example, imagine a plan
that looks like this:
Nested Loop
-> Parallel Seq Scan on p
-> Index Scan on q
Index Scan: q.x = p.x
The parallel workers should fill up the tuple queues used by the
parallel seq scan so that the master doesn't have to do any of that
work itself. Therefore, the normal situation will be that those tuple
queues are all full. If an error occurs in a worker at that point, it
can't add it to the tuple queue, because the tuple queue is full. But
even if it could do that, the master then won't notice the error until
it reads all of the queued-up tuple messges that are in the queue in
front of the error, and maybe some messages from the other queues as
well, since it probably round-robins between the queues or something
like that. Basically, it could do a lot of extra work before noticing
that error in there.
Now we could avoid that by having the master read messages from the
queue immediately and just save them off to local storage if they
aren't error messages. But that's not very desirable either, because
now we have no flow-control. The workers will just keep spamming
tuples that the master isn't ready for into the queues, and the master
will keep reading them and saving them to local storage, and
eventually it will run out of memory and die.
We could engineer some solution to this problem, of course, but it
seems quite a bit simpler to just have two queues. The error queues
don't need to be very big (I made them 16kB, which is trivial on any
system on which you care about having working parallelism) and the
tuple queues can be sized as needed to avoid pipeline stalls.
2.
Currently there is no interface for wait_for_workers_to_become_ready()
in your patch, don't you think it is important that before start of fetching
tuples, we should make sure all workers are started, what if some worker
fails to start?
I think that, in general, getting the most benefit out of parallelism
means *avoiding* situations where backends have to wait for each
other. If the relation being scanned is not too large, the user
backend might be able to finish the whole scan - or a significant
fraction of it - before the workers initialize. Of course in that
case it might have been a bad idea to parallelize in the first place,
but we should still try to make the best of the situation. If some
worker fails to start, then instead of having the full degree N
parallelism we were hoping for, we have some degree K < N, so things
will take a little longer, but everything should still work.
--
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 3, 2015 at 7:31 PM, Andres Freund <andres@2ndquadrant.com> wrote:
A couple remarks:
* Shouldn't this provide more infrastructure to deal with the fact that
we might get less parallel workers than we had planned for?
Maybe, but I'm not really sure what that should look like. My working
theory is that individual parallel applications (executor nodes, or
functions that use parallelism internally, or whatever) should be
coded in such a way that they work correctly even if the number of
workers that starts is smaller than what they requested, or even zero.
It may turn out that this is impractical in some cases, but I don't
know what those cases are yet so I don't know what the common threads
will be.
I think parallel seq scan should work by establishing N tuple queues
(where N is the number of workers we have). When asked to return a
tuple, the master should poll all of those queues one after another
until it finds one that contains a tuple. If none of them contain a
tuple then it should claim a block to scan and return a tuple from
that block. That way, if there are enough running workers to keep up
with the master's demand for tuples, the master won't do any of the
actual scan itself. But if the workers can't keep up -- e.g. suppose
90% of the CPU consumed by the query is in the filter qual for the
scan -- then the master can pitch in along with everyone else. As a
non-trivial fringe benefit, if the workers don't all start, or take a
while to initialize, the user backend isn't stalled meanwhile.
* I wonder if parallel contexts shouldn't be tracked via resowners
That is a good question. I confess that I'm somewhat fuzzy about
which things should be tracked via the resowner mechanism vs. which
things should have their own bespoke bookkeeping. However, the
AtEOXact_Parallel() stuff happens well before ResourceOwnerRelease(),
which makes merging them seem not particularly clean.
* combocid.c should error out in parallel mode, as insurance
Eh, which function? HeapTupleHeaderGetCmin(),
HeapTupleHeaderGetCmax(), and AtEOXact_ComboCid() are intended to work
in parallel mode. HeapTupleHeaderAdjustCmax() could
Assert(!IsInParallelMode()) but the only calls are in heap_update()
and heap_delete() which already have error checks, so putting yet
another elog() there seems like overkill.
* I doubt it's a good idea to allow heap_insert, heap_inplace_update,
index_insert. I'm not convinced that those will be handled correct and
relaxing restrictions is easier than adding them.
I'm fine with adding checks to heap_insert() and
heap_inplace_update(). I'm not sure we really need to add anything to
index_insert(); how are we going to get there without hitting some
other prohibited operation first?
* I'd much rather separate HandleParallelMessageInterrupt() in one
variant that just tells the machinery there's a interrupt (called from
signal handlers) and one that actually handles them. We shouldn't even
consider adding any new code that does allocations, errors and such in
signal handlers. That's just a *bad* idea.
That's a nice idea, but I didn't invent the way this crap works today.
ImmediateInterruptOK gets set to true while performing a lock wait,
and we need to be able to respond to errors while in that state. I
think there's got to be a better way to handle that than force every
asynchronous operation to have to cope with the fact that
ImmediateInterruptOK may be set or not set, but as of today that's
what you have to do.
* I'm not a fan of the shm_toc stuff... Verbose and hard to read. I'd
much rather place a struct there and be careful about not using
pointers. That also would obliviate the need to reserve some ids.
I don't see how that's going to work with variable-size data
structures, and a bunch of the things that we need to serialize are
variable-size. Moreover, I'm not really interested in rehashing this
design again. I know it's not your favorite; you've said that before.
But it makes it possible to write code to do useful things in
parallel, a capability that we do not otherwise have. And I like it
fine.
* Doesn't the restriction in GetSerializableTransactionSnapshotInt()
apply for repeatable read just the same?
Yeah. I'm not sure whether we really need that check at all, because
there is a check in GetTransactionSnapshot() that probably checks the
same thing.
* I'm not a fan of relying on GetComboCommandId() to restore things in
the same order as before.
I thought that was a little wonky, but it's not likely to break, and
there is an elog(ERROR) there to catch it if it does, so I'd rather
not make it more complicated.
* I'd say go ahead and commit the BgworkerByOid thing
earlier/independently. I've seen need for that a couple times.
Woohoo! I was hoping someone would say that.
* s/entrypints/entrypoints/
Thanks.
--
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 Sat, Jan 3, 2015 at 7:31 PM, Andres Freund <andres@2ndquadrant.com> wrote:
* I wonder if parallel contexts shouldn't be tracked via resowners
That is a good question. I confess that I'm somewhat fuzzy about
which things should be tracked via the resowner mechanism vs. which
things should have their own bespoke bookkeeping. However, the
AtEOXact_Parallel() stuff happens well before ResourceOwnerRelease(),
which makes merging them seem not particularly clean.
FWIW, the resowner mechanism was never meant as a substitute for bespoke
bookkeeping. What it is is a helper mechanism to reduce the need for
PG_TRY blocks that guarantee that a resource-releasing function will be
called even in error paths. I'm not sure whether that analogy applies
well in parallel-execution cases.
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 22 December 2014 at 19:14, Robert Haas <robertmhaas@gmail.com> wrote:
Here is another new version, with lots of bugs fixed.
An initial blind review, independent of other comments already made on thread.
OK src/backend/access/heap/heapam.c
heapam.c prohibitions on update and delete look fine
OK src/backend/access/transam/Makefile
OK src/backend/access/transam/README.parallel
README.parallel and all concepts look good
PARTIAL src/backend/access/transam/parallel.c
wait_patiently mentioned in comment but doesn’t exist
Why not make nworkers into a uint?
Trampoline? Really? I think we should define what we mean by that,
somewhere, rather than just use the term as if it was self-evident.
Entrypints?
..
OK src/backend/access/transam/varsup.c
QUESTIONS src/backend/access/transam/xact.c
These comments don’t have any explanation or justification
+ * This stores XID of the toplevel transaction to which we belong.
+ * Normally, it's the same as TopTransactionState.transactionId, but in
+ * a parallel worker it may not be.
+ * If we're a parallel worker, there may be additional XIDs that we need
+ * to regard as part of our transaction even though they don't appear
+ * in the transaction state stack.
This comment is copied-and-pasted too many times for safety and elegance
+ /*
+ * Workers synchronize transaction state at the beginning of
each parallel
+ * operation, so we can't account for transaction state change
after that
+ * point. (Note that this check will certainly error out if
s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an
invalid case
+ * below.)
+ */
We need a single Check() that contains more detail and comments within
Major comments
* Parallel stuff at start sounded OK
* Transaction stuff strikes me as overcomplicated and error prone.
Given there is no explanation or justification for most of it, I’d be
inclined to question why its there
* If we have a trampoline for DSM, why don’t we use a trampoline for
snapshots, then you wouldn’t need to do all this serialize stuff
This is very impressive, but it looks like we’re trying to lift too
much weight on the first lift. If we want to go anywhere near this, we
need to have very clear documentation about how and why its like that.
I’m actually very sorry to say that because the review started very
well, much much better than most.
--
Simon Riggs 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 Mon, Jan 5, 2015 at 9:57 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Sat, Jan 3, 2015 at 7:31 PM, Andres Freund <andres@2ndquadrant.com>
wrote:
* Doesn't the restriction in GetSerializableTransactionSnapshotInt()
apply for repeatable read just the same?Yeah. I'm not sure whether we really need that check at all, because
there is a check in GetTransactionSnapshot() that probably checks the
same thing.
The check in GetSerializableTransactionSnapshotInt() will also prohibit
any user/caller of SetSerializableTransactionSnapshot() in parallel mode
as that can't be prohibited by GetTransactionSnapshot().
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Mon, Jan 5, 2015 at 6:21 PM, Simon Riggs <simon@2ndquadrant.com> wrote:
PARTIAL src/backend/access/transam/parallel.c
wait_patiently mentioned in comment but doesn’t exist
Woops. Fixed. Originally, DestroyParallelContext() had a Boolean
argument wait_patiently, specifying whether it should exit at once or
do the equivalent of WaitForParallelWorkersToFinish() first. But
that turned out not to work well - for example, in the included
parallel_count example, it's very important to (1) first wait for all
the workers to exit, (2) then read the final count, and (3) only after
that destroy the dynamic shared memory segment. So
WaitForParallelWorkersToFinish() got separated out into a different
function, but I failed to update the comments.
Why not make nworkers into a uint?
We don't have or use a type called uint. I could make it uint32 or
uint64 or uint16, but I don't see much advantage in that. I could
also make it unsigned, but we make very little use of the unsigned
type generally, so I picked int as most consistent with our general
practice. If there's a consensus that something else is much better
than int, I will change it throughout, but I think we have bigger fish
to fry.
Trampoline? Really? I think we should define what we mean by that,
somewhere, rather than just use the term as if it was self-evident.
Comments improved.
Entrypints?
Already noted by Andres; fixed in the attached version.
These comments don’t have any explanation or justification
OK, I rewrote them. Hopefully it's better now.
This comment is copied-and-pasted too many times for safety and elegance
It's not the same comment each time it appears; it appears in the
exact form you quoted just twice. It does get a little long-winded, I
guess, but I think that parallelism is a sufficiently-unfamiliar
concept to us as a group that it makes sense to have a detailed
comment in each place explaining exactly why that particular callsite
requires a check, so that's what I've tried to do.
* Parallel stuff at start sounded OK
* Transaction stuff strikes me as overcomplicated and error prone.
Given there is no explanation or justification for most of it, I’d be
inclined to question why its there
Gosh, I was pretty pleased with how simple the transaction integration
turned out to be. Most of what's there right now is either (a) code
to copy state from the master to the parallel workers or (b) code to
throw errors if the workers try to things that aren't safe. I suspect
there are a few things missing, but I don't see anything there that
looks unnecessary.
* If we have a trampoline for DSM, why don’t we use a trampoline for
snapshots, then you wouldn’t need to do all this serialize stuff
The trampoline is just to let extensions use this infrastructure if
they want to; there's no way to avoid the serialization-and-restore
stuff unless we switch to creating the child processes using fork(),
but that would:
1. Not work on Windows.
2. Require the postmaster to deal with processes that are not its
immediate children.
3. Possibly introduce other bugs.
Since I've spent a year and a half trying to get this method to work
and am now, I think, almost there, I'm not particularly sanguine about
totally changing the approach.
This is very impressive, but it looks like we’re trying to lift too
much weight on the first lift. If we want to go anywhere near this, we
need to have very clear documentation about how and why its like that.
I’m actually very sorry to say that because the review started very
well, much much better than most.
When I posted the group locking patch, it got criticized because it
didn't actually do anything useful by itself; similarly, the
pg_background patch was criticized for not being a large enough step
toward parallelism. So, this time, I posted something more
comprehensive. I don't think it's quite complete yet. I expect a
committable version of this patch to be maybe another 500-1000 lines
over what I have here right now -- I think it needs to do something
about heavyweight locking, and I expect that there are some unsafe
things that aren't quite prohibited yet. But the current patch is
only 2300 lines, which is not astonishingly large for a feature of
this magnitude; if anything, I'd say it's surprisingly small, due to a
year and a half of effort laying the necessary groundwork via long
series of preliminary commits. I'm not unwilling to divide this up
some more if we can agree on a way to do that that makes sense, but I
think we're nearing the point where we need to take the plunge and
say, look, this is version one of parallelism. Thunk.
In addition to the changes mentioned above, the attached version
prohibits a few more things (as suggested by Andres) and passes the
dsm_segment to the user-supplied entrypoint (as requested off-list by
Andres, because otherwise you can't set up additional shm_mq
structures).
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v0.3.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v0.3.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..3c0ae7d
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,12 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_sleep(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
+
+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_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..0a32ea8
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,238 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.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/snapmgr.h"
+#include "utils/tqual.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_sleep);
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct
+{
+ int32 sleep_time;
+} ParallelSleepInfo;
+
+typedef struct
+{
+ int32 relid;
+ slock_t mutex;
+ BlockNumber lastblock;
+ BlockNumber currentblock;
+ int64 ntuples;
+} ParallelCountInfo;
+
+void _PG_init(void);
+void sleep_worker_main(dsm_segment *seg, shm_toc *toc);
+void count_worker_main(dsm_segment *seg, shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelCountInfo *info);
+
+Datum
+parallel_sleep(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelSleepInfo *info;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy",
+ "sleep_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelSleepInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelSleepInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ WaitForParallelWorkersToFinish(pcxt);
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelCountInfo *info;
+ Relation rel;
+ int64 result;
+
+ 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_dummy",
+ "count_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelCountInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelCountInfo));
+ info->relid = relid;
+ SpinLockInit(&info->mutex);
+ info->lastblock = RelationGetNumberOfBlocks(rel);
+ info->currentblock = 0;
+ info->ntuples = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, info);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ result = info->ntuples;
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(result);
+}
+
+void
+sleep_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelSleepInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
+
+void
+count_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelCountInfo *info;
+ Relation rel;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ rel = relation_open(info->relid, AccessShareLock);
+ count_helper(rel, info);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelCountInfo *info)
+{
+ int64 ntuples = 0;
+ int64 mytuples = 0;
+ Oid relid = info->relid;
+ Snapshot snapshot = GetActiveSnapshot();
+
+ for (;;)
+ {
+ 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);
+ lines = PageGetMaxOffsetNumber(page);
+ ntuples = 0;
+
+ all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
+
+ for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
+ lineoff <= lines;
+ lineoff++, lpp++)
+ {
+ HeapTupleData loctup;
+
+ if (!ItemIdIsNormal(lpp))
+ continue;
+ if (all_visible)
+ {
+ ++ntuples;
+ continue;
+ }
+
+ loctup.t_tableOid = relid;
+ loctup.t_data = (HeapTupleHeader) PageGetItem(page, lpp);
+ loctup.t_len = ItemIdGetLength(lpp);
+
+ if (HeapTupleSatisfiesVisibility(&loctup, snapshot, buffer))
+ ++ntuples;
+ }
+
+ UnlockReleaseBuffer(buffer);
+ }
+
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 61216e5..8e3ccf4 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 24e300c..eee1ca0 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..e427275
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,36 @@
+Overview
+========
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). These functions
+don't launch any workers or directly enable parallelism, but they put in place
+a variety of prohibitions required to make parallelism safe.
+
+To actually parallelize a particular operation, use a ParallelContext. This
+establishes a dynamic shared memory segment and registers dynamic background
+workers which will attach to that segment. We arrange to synchronize various
+pieces of state - such as, most simply, the database and user OIDs - from the
+backend that is initiating parallelism to all of the background workers
+launched via a ParallelContext. The basic coding pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..71374cc
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,886 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 6);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ * We do, however, give up on registering any more workers once
+ * registration fails the first time; no sense beating our head against
+ * a brick wall.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed and the dynamic shared memory segment
+ * is immediately unmapped.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill and forget about each worker in turn. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ {
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].bgwhandle);
+ }
+ if (pcxt->worker[i].error_mqh != NULL)
+ pfree(pcxt->worker[i].error_mqh);
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler)
+ && InterruptHoldoffCount == 0 && CritSectionCount == 0
+ && !HandlingParallelMessages)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+ HandlingParallelMessages = true;
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ HandlingParallelMessages = false;
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read messages for as long as we have an error queue; if we
+ * have hit (or hit while reading) ReadyForQuery, this will go to
+ * NULL.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else if (res == SHM_MQ_DETACHED)
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker is insufficient justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * XXX. We should probably use the error context callbacks in
+ * effect at the time the parallel context was created.
+ */
+
+ ThrowErrorData(&edata);
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index c541156..92f657e 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8b2f714..76aeef7 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,30 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only the top frame of the transaction state stack is copied to a parallel
+ * worker, but GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +137,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +179,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +210,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +380,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +395,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +489,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +547,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +680,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +783,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +866,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +926,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1837,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1968,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1845,6 +1982,10 @@ CommitTransaction(void)
TransStateAsString(s->state));
Assert(s->parent == NULL);
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/*
* Do pre-commit processing that involves calling user-defined code, such
* as triggers. Since closing cursors could queue trigger actions,
@@ -1867,7 +2008,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1915,9 +2057,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2090,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2139,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2165,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2191,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2437,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2458,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2507,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2384,10 +2542,22 @@ AbortTransaction(void)
AtAbort_Twophase();
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2575,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2599,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2652,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2709,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2745,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3033,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3423,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3601,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3704,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3743,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3775,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3810,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3845,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3923,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3959,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4048,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4084,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4117,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4151,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4170,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4246,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4338,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4157,6 +4437,13 @@ CommitSubTransaction(void)
elog(WARNING, "CommitSubTransaction while in %s state",
TransStateAsString(s->state));
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Pre-commit processing goes here */
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
@@ -4315,6 +4602,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4749,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4797,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4994,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index ed2b05a..5398b70 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -476,7 +476,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8b1c727..0cc35b4 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 811e1d4..d9b0aed 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 8c799d3..9223f5e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 4d11260..62b615a 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index cfa4a24..72d55c7 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 6e6b429..b07978c 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 675f985..637749a 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -471,7 +471,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1664,7 +1664,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..451f814 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5305,7 +5305,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d953545..7b7b57a 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index cd9a287..0678678 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index e783955..b35cbbd 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..4285748 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -37,6 +37,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3704,7 +3706,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 71580e8..623f985 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 8fccb4c..a045062 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -350,11 +350,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -365,7 +364,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index c348034..8f6d517 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 77c3494..6f2a571 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index ea7a905..33b0b52 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids || !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index d601efe..bc071aa 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,13 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot update SecondarySnapshpt during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +372,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +420,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +584,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1296,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1399,162 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ *
+ * If set_transaction_snapshot is true, the snapshot is additionally installed
+ * as the transaction snapshot.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..b651218
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,59 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index b018aa4..91d9d73 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index f8e68c9..78583b3 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 1558a75..fb7754f 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -258,6 +258,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -290,7 +291,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -391,7 +392,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a3b3d5f..273cecb 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -130,6 +130,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 0c4611b..9f4e4b9 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index c625562..3046e52 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index 9e482ff..232f729 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index abe7016..7efd427 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On 6 January 2015 at 16:33, Robert Haas <robertmhaas@gmail.com> wrote:
These comments don’t have any explanation or justification
OK, I rewrote them. Hopefully it's better now.
Thanks for new version and answers.
* Transaction stuff strikes me as overcomplicated and error prone.
Given there is no explanation or justification for most of it, I’d be
inclined to question why its thereGosh, I was pretty pleased with how simple the transaction integration
turned out to be. Most of what's there right now is either (a) code
to copy state from the master to the parallel workers or (b) code to
throw errors if the workers try to things that aren't safe. I suspect
there are a few things missing, but I don't see anything there that
looks unnecessary.
If you can explain it in more detail in comments and README, I may
agree. At present, I don't get it and it makes me nervous.
The comment says
"Only the top frame of the transaction state stack is copied to a parallel
worker"
but I'm not sure why.
Top meaning the current subxact or the main xact?
If main, why are do we need XactTopTransactionId
This is very impressive, but it looks like we’re trying to lift too
much weight on the first lift. If we want to go anywhere near this, we
need to have very clear documentation about how and why its like that.
I’m actually very sorry to say that because the review started very
well, much much better than most.When I posted the group locking patch, it got criticized because it
didn't actually do anything useful by itself; similarly, the
pg_background patch was criticized for not being a large enough step
toward parallelism. So, this time, I posted something more
comprehensive. I don't think it's quite complete yet. I expect a
committable version of this patch to be maybe another 500-1000 lines
over what I have here right now -- I think it needs to do something
about heavyweight locking, and I expect that there are some unsafe
things that aren't quite prohibited yet. But the current patch is
only 2300 lines, which is not astonishingly large for a feature of
this magnitude; if anything, I'd say it's surprisingly small, due to a
year and a half of effort laying the necessary groundwork via long
series of preliminary commits. I'm not unwilling to divide this up
some more if we can agree on a way to do that that makes sense, but I
think we're nearing the point where we need to take the plunge and
say, look, this is version one of parallelism. Thunk.
I want this also; the only debate is where to draw the line and please
don't see that as criticism.
I'm very happy it's so short, I agree it could be longer.
--
Simon Riggs 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, Jan 6, 2015 at 3:04 PM, Simon Riggs <simon@2ndquadrant.com> wrote:
If you can explain it in more detail in comments and README, I may
agree. At present, I don't get it and it makes me nervous.The comment says
"Only the top frame of the transaction state stack is copied to a parallel
worker"
but I'm not sure why.Top meaning the current subxact or the main xact?
If main, why are do we need XactTopTransactionId
Current subxact.
I initially thought of copying the entire TransactionStateData stack,
but Heikki suggested (via IM) that I do it this way instead. I
believe his concern was that it's never valid to commit or roll back
to a subtransaction that is not at the top of the stack, and if you
don't copy the stack, you avoid the risk of somehow ending up in that
state. Also, you avoid having to invent resource owners for
(sub)transactions that don't really exist in the current process. On
the other hand, you do end up with a few special cases that wouldn't
exist with the other approach. Still, I'm pretty happy to have taken
Heikki's advice: it was certainly simple to implement this way, plus
hopefully that way at least one person likes what I ended up with.
:-)
What else needs clarification?
--
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 6 January 2015 at 21:01, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Jan 6, 2015 at 3:04 PM, Simon Riggs <simon@2ndquadrant.com> wrote:
If you can explain it in more detail in comments and README, I may
agree. At present, I don't get it and it makes me nervous.The comment says
"Only the top frame of the transaction state stack is copied to a parallel
worker"
but I'm not sure why.Top meaning the current subxact or the main xact?
If main, why are do we need XactTopTransactionIdCurrent subxact.
TopTransactionStateData points to the top-level transaction data, but
XactTopTransactionId points to the current subxact.
So when you say "Only the top frame of the transaction state stack is
copied" you don't mean the top, you mean the bottom (the latest
subxact)? Which then becomes the top in the parallel worker? OK...
I initially thought of copying the entire TransactionStateData stack,
but Heikki suggested (via IM) that I do it this way instead. I
believe his concern was that it's never valid to commit or roll back
to a subtransaction that is not at the top of the stack, and if you
don't copy the stack, you avoid the risk of somehow ending up in that
state. Also, you avoid having to invent resource owners for
(sub)transactions that don't really exist in the current process. On
the other hand, you do end up with a few special cases that wouldn't
exist with the other approach. Still, I'm pretty happy to have taken
Heikki's advice: it was certainly simple to implement this way, plus
hopefully that way at least one person likes what I ended up with.
:-)What else needs clarification?
Those comments really belong in a README, not the first visible
comment in xact.c
You need to start with the explanation that parallel workers have a
faked-up xact stack to make it easier to copy and manage. That is
valid because we never change xact state during a worker operation.
I get it now and agree, but please work some more on clarity of
README, comments and variable naming!
--
Simon Riggs 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 1/6/15, 10:33 AM, Robert Haas wrote:
Entrypints?
Already noted by Andres; fixed in the attached version.
Perhaps we only parallelize while drinking beer... ;)
CreateParallelContext(): Does it actually make sense to have nworkers=0? ISTM that's a bogus case. Also, since the number of workers will normally be determined dynamically by the planner, should that check be a regular conditional instead of just an Assert?
In LaunchParallelWorkers() the "Start Workers" comment states that we give up registering more workers if one fails to register, but there's nothing in the if condition to do that, and I don't see RegisterDynamicBackgroundWorker() doing it either. Is the comment just incorrect?
SerializeTransactionState(): instead of looping through the transaction stack to calculate nxids, couldn't we just set it to maxsize - sizeof(TransactionId) * 3? If we're looping a second time for safety a comment explaining that would be useful...
sequence.c: Is it safe to read a sequence value in a parallel worker if the cache_value is > 1?
This may be a dumb question, but for functions do we know that all pl's besides C and SQL use SPI? If not I think they could end up writing in a worker.
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
ISTM it'd be good to leave that comment in place (after the if).
RestoreComboCIDState(): Assert(!comboCids || !comboHash): Shouldn't that be &&? AIUI both should always be either set or 0.
Typo: + elog(ERROR, "cannot update SecondarySnapshpt during a parallel operation");
The comment for RestoreSnapshot refers to set_transaction_snapshot, but that doesn't actually exist (or isn't referenced).
--
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 6 January 2015 at 21:37, Simon Riggs <simon@2ndquadrant.com> wrote:
I get it now and agree
Yes, very much.
Should we copy both the top-level frame and the current subxid?
Hot Standby links subxids directly to the top-level, so this would be similar.
If we copied both, we wouldn't need to special case the Get functions.
It would still be O(1).
but please work some more on clarity of
README, comments and variable naming!
Something other than "top" sounds good.
--
Simon Riggs 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, Jan 6, 2015 at 4:37 PM, Simon Riggs <simon@2ndquadrant.com> wrote:
So when you say "Only the top frame of the transaction state stack is
copied" you don't mean the top, you mean the bottom (the latest
subxact)? Which then becomes the top in the parallel worker? OK...
The item most recently added to the stack is properly called the top,
but I guess it's confusing in this case because the item on the bottom
of the stack is referred to as the TopTransaction. I'll see if I can
rephrase that.
Those comments really belong in a README, not the first visible
comment in xact.c
OK.
You need to start with the explanation that parallel workers have a
faked-up xact stack to make it easier to copy and manage. That is
valid because we never change xact state during a worker operation.
OK.
--
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 7 January 2015 at 13:11, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Jan 6, 2015 at 4:37 PM, Simon Riggs <simon@2ndquadrant.com> wrote:
So when you say "Only the top frame of the transaction state stack is
copied" you don't mean the top, you mean the bottom (the latest
subxact)? Which then becomes the top in the parallel worker? OK...The item most recently added to the stack is properly called the top,
but I guess it's confusing in this case because the item on the bottom
of the stack is referred to as the TopTransaction. I'll see if I can
rephrase that.
Yes, I didn't mean to suggest the confusion was introduced by you -
it's just you stepped into the mess by correctly using the word top in
a place where its meaning would be opposite to the close-by usage of
TopTransaction.
Anyway, feeling good about this now. Thanks for your patience.
--
Simon Riggs 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, Jan 6, 2015 at 9:37 PM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
CreateParallelContext(): Does it actually make sense to have nworkers=0?
ISTM that's a bogus case.
I'm not sure whether we'll ever use the zero-worker case in
production, but I've certainly found it useful for
performance-testing.
Also, since the number of workers will normally be
determined dynamically by the planner, should that check be a regular
conditional instead of just an Assert?
I don't think that's really necessary. It shouldn't be too much of a
stretch for the planner to come up with a non-negative value.
In LaunchParallelWorkers() the "Start Workers" comment states that we give
up registering more workers if one fails to register, but there's nothing in
the if condition to do that, and I don't see
RegisterDynamicBackgroundWorker() doing it either. Is the comment just
incorrect?
Woops, that got changed at some point and I forgot to update the
comment. Will fix.
SerializeTransactionState(): instead of looping through the transaction
stack to calculate nxids, couldn't we just set it to maxsize -
sizeof(TransactionId) * 3? If we're looping a second time for safety a
comment explaining that would be useful...
Yeah, I guess we could do that. I don't think it really matters very
much one way or the other.
sequence.c: Is it safe to read a sequence value in a parallel worker if the
cache_value is > 1?
No, because the sequence cache isn't synchronized between the workers.
Maybe it would be safe if cache_value == 1, but there's not much
use-case: how often are you going to have a read-only query that uses
a sequence value. At some point we can look at making sequences
parallel-safe, but worrying about it right now doesn't seem like a
good use of time.
This may be a dumb question, but for functions do we know that all pl's
besides C and SQL use SPI? If not I think they could end up writing in a
worker.
Well, the lower-level checks would catch that. But it is generally
true that there's no way to prevent arbitrary C code from doing things
that are unsafe in parallel mode and that we can't tell are unsafe.
As I've said before, I think that we'll need to have a method of
labeling functions as parallel-safe or not, but this patch isn't
trying to solve that part of the problem.
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void) errmsg("canceling statement due to user request"))); } } - /* If we get here, do nothing (probably, QueryCancelPending was reset) */ + if (ParallelMessagePending) + HandleParallelMessageInterrupt(false); ISTM it'd be good to leave that comment in place (after the if).RestoreComboCIDState(): Assert(!comboCids || !comboHash): Shouldn't that be
&&? AIUI both should always be either set or 0.
Fixed.
Typo: + elog(ERROR, "cannot update SecondarySnapshpt during a
parallel operation");
Fixed.
The comment for RestoreSnapshot refers to set_transaction_snapshot, but that
doesn't actually exist (or isn't referenced).
Fixed.
Will post a new version in a bit.
--
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, Jan 7, 2015 at 10:34 AM, Simon Riggs <simon@2ndquadrant.com> wrote:
Yes, I didn't mean to suggest the confusion was introduced by you -
it's just you stepped into the mess by correctly using the word top in
a place where its meaning would be opposite to the close-by usage of
TopTransaction.Anyway, feeling good about this now. Thanks for your patience.
Thanks for the kind words, and the review. Here's a new version with
a much-expanded README and some other changes to the comments to
hopefully address some of your other concerns. I also fixed a couple
of other problems while I was at it: the comments in xact.c claimed
that parallel shutdown waits for workers to exit, but parallel.c
didn't know anything about that. Also, I fixed it so that when an
error is propagated from the parallel worker to the user backend, the
error context in effect at the time the parallel context was created
is used, rather than whatever is in effect when we notice the error.
I have little doubt that this version is still afflicted with various
bugs, and the heavyweight locking issue remains to be dealt with, but
on the whole I think this is headed in the right direction.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v0.4.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v0.4.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..3c0ae7d
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,12 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_sleep(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
+
+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_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..0a32ea8
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,238 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.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/snapmgr.h"
+#include "utils/tqual.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_sleep);
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct
+{
+ int32 sleep_time;
+} ParallelSleepInfo;
+
+typedef struct
+{
+ int32 relid;
+ slock_t mutex;
+ BlockNumber lastblock;
+ BlockNumber currentblock;
+ int64 ntuples;
+} ParallelCountInfo;
+
+void _PG_init(void);
+void sleep_worker_main(dsm_segment *seg, shm_toc *toc);
+void count_worker_main(dsm_segment *seg, shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelCountInfo *info);
+
+Datum
+parallel_sleep(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelSleepInfo *info;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy",
+ "sleep_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelSleepInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelSleepInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ WaitForParallelWorkersToFinish(pcxt);
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelCountInfo *info;
+ Relation rel;
+ int64 result;
+
+ 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_dummy",
+ "count_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelCountInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelCountInfo));
+ info->relid = relid;
+ SpinLockInit(&info->mutex);
+ info->lastblock = RelationGetNumberOfBlocks(rel);
+ info->currentblock = 0;
+ info->ntuples = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, info);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ result = info->ntuples;
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(result);
+}
+
+void
+sleep_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelSleepInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
+
+void
+count_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelCountInfo *info;
+ Relation rel;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ rel = relation_open(info->relid, AccessShareLock);
+ count_helper(rel, info);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelCountInfo *info)
+{
+ int64 ntuples = 0;
+ int64 mytuples = 0;
+ Oid relid = info->relid;
+ Snapshot snapshot = GetActiveSnapshot();
+
+ for (;;)
+ {
+ 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);
+ lines = PageGetMaxOffsetNumber(page);
+ ntuples = 0;
+
+ all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
+
+ for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
+ lineoff <= lines;
+ lineoff++, lpp++)
+ {
+ HeapTupleData loctup;
+
+ if (!ItemIdIsNormal(lpp))
+ continue;
+ if (all_visible)
+ {
+ ++ntuples;
+ continue;
+ }
+
+ loctup.t_tableOid = relid;
+ loctup.t_data = (HeapTupleHeader) PageGetItem(page, lpp);
+ loctup.t_len = ItemIdGetLength(lpp);
+
+ if (HeapTupleSatisfiesVisibility(&loctup, snapshot, buffer))
+ ++ntuples;
+ }
+
+ UnlockReleaseBuffer(buffer);
+ }
+
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 61216e5..8e3ccf4 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 24e300c..eee1ca0 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..401cfd6
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,206 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..9f3a01a
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,921 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 6);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler)
+ && InterruptHoldoffCount == 0 && CritSectionCount == 0
+ && !HandlingParallelMessages)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+ HandlingParallelMessages = true;
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ HandlingParallelMessages = false;
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read messages for as long as we have an error queue; if we
+ * have hit (or hit while reading) ReadyForQuery, this will go to
+ * NULL.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else if (res == SHM_MQ_DETACHED)
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback *save_error_context_stack;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ error_context_stack = pcxt->error_context_stack;
+ ThrowErrorData(&edata);
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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));
+ if (seg == NULL)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("unable to map dynamic shared memory segment")));
+ toc = shm_toc_attach(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index c541156..92f657e 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 8b2f714..d27d184 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,32 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only a single TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +139,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +181,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +212,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +382,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +397,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +491,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +549,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +682,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +785,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +868,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +928,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1839,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1970,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1845,6 +1984,10 @@ CommitTransaction(void)
TransStateAsString(s->state));
Assert(s->parent == NULL);
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/*
* Do pre-commit processing that involves calling user-defined code, such
* as triggers. Since closing cursors could queue trigger actions,
@@ -1867,7 +2010,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1915,9 +2059,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2092,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2141,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2167,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2193,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2439,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2460,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,12 +2509,20 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
Assert(s->parent == NULL);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* set the current transaction state information appropriately during the
* abort processing
*/
@@ -2385,9 +2552,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2577,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2601,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2654,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2711,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2747,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3035,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3425,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3603,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3706,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3745,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3777,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3812,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3847,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3925,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3961,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4050,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4086,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4119,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4153,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4172,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4248,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4340,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4157,6 +4439,13 @@ CommitSubTransaction(void)
elog(WARNING, "CommitSubTransaction while in %s state",
TransStateAsString(s->state));
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Pre-commit processing goes here */
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
@@ -4315,6 +4604,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4751,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4799,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4996,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index ed2b05a..5398b70 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -476,7 +476,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8b1c727..0cc35b4 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 811e1d4..d9b0aed 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 8c799d3..9223f5e 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 4d11260..62b615a 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index cfa4a24..72d55c7 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 6e6b429..b07978c 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 675f985..637749a 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -471,7 +471,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1664,7 +1664,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 85a3b3a..b5a70d7 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -984,6 +984,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index 5106f52..451f814 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5305,7 +5305,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index d953545..7b7b57a 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index cd9a287..0678678 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index e783955..b35cbbd 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index cc62b2c..4285748 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -37,6 +37,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2968,7 +2969,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3704,7 +3706,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 71580e8..623f985 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 8fccb4c..a045062 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -350,11 +350,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -365,7 +364,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index c348034..8f6d517 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 77c3494..6f2a571 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index ea7a905..6c8056e 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index d601efe..11ec825 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +373,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +421,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +585,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1297,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1400,159 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..761ba1f
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,61 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index b018aa4..91d9d73 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index f8e68c9..78583b3 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 1558a75..fb7754f 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -258,6 +258,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -290,7 +291,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -391,7 +392,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a3b3d5f..a270da4 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
@@ -130,6 +132,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 0c4611b..9f4e4b9 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index c625562..3046e52 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index 9e482ff..232f729 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index abe7016..7efd427 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On 1/7/15, 9:39 AM, Robert Haas wrote:
sequence.c: Is it safe to read a sequence value in a parallel worker if the
cache_value is > 1?
No, because the sequence cache isn't synchronized between the workers.
Maybe it would be safe if cache_value == 1, but there's not much
use-case: how often are you going to have a read-only query that uses
a sequence value. At some point we can look at making sequences
parallel-safe, but worrying about it right now doesn't seem like a
good use of time.
Agreed, I was more concerned with calls to nextval(), which don't seem to be prevented in parallel mode?
This may be a dumb question, but for functions do we know that all pl's
besides C and SQL use SPI? If not I think they could end up writing in a
worker.Well, the lower-level checks would catch that. But it is generally
true that there's no way to prevent arbitrary C code from doing things
that are unsafe in parallel mode and that we can't tell are unsafe.
As I've said before, I think that we'll need to have a method of
labeling functions as parallel-safe or not, but this patch isn't
trying to solve that part of the problem.
I was more thinking about all the add-on pl's like pl/ruby. But yeah, I don't see that there's much we can do if they're not using SPI.
--
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 7, 2015 at 11:03 PM, Robert Haas <robertmhaas@gmail.com> wrote:
I have little doubt that this version is still afflicted with various
bugs, and the heavyweight locking issue remains to be dealt with, but
on the whole I think this is headed in the right direction.
+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));
Here, I think DatumGetUInt32() needs to be used instead of
DatumGetInt32() as the segment handle is uint32.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Jan 7, 2015 at 3:54 PM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
Agreed, I was more concerned with calls to nextval(), which don't seem to be
prevented in parallel mode?
It looks prevented:
/*
* Forbid this during parallel operation because, to make it work,
* the cooperating backends would need to share the backend-local cached
* sequence information. Currently, we don't support that.
*/
PreventCommandIfParallelMode("nextval()");
I was more thinking about all the add-on pl's like pl/ruby. But yeah, I
don't see that there's much we can do if they're not using SPI.
Actually, there is: it's forbidden at the layer of heap_insert(),
heap_update(), heap_delete(). It's hard to imagine anyone trying to
modify the database as a lower level than 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 Thu, Jan 8, 2015 at 6:52 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
+ seg = dsm_attach(DatumGetInt32(main_arg));
Here, I think DatumGetUInt32() needs to be used instead of
DatumGetInt32() as the segment handle is uint32.
OK, I'll change that in the next version.
--
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 13, 2015 at 1:33 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Jan 8, 2015 at 6:52 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
+ seg = dsm_attach(DatumGetInt32(main_arg));
Here, I think DatumGetUInt32() needs to be used instead of
DatumGetInt32() as the segment handle is uint32.OK, I'll change that in the next version.
No issues, I have another question related to below code:
+HandleParallelMessages(void)
+{
..
..
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read messages for as long as we have an error queue; if we
+ * have hit (or hit while reading) ReadyForQuery, this will go to
+ * NULL.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_SUCCESS)
Here we are checking the error queue for all the workers and this loop
will continue untill all have sent ReadyForQuery() message ('Z') which
will make this loop continue till all workers have finished their work.
Assume situation where first worker has completed the work and sent
'Z' message and second worker is still sending some tuples, now above
code will keep on waiting for 'Z' message from second worker and won't
allow to receive tuples sent by second worker till it send 'Z' message.
As each worker send its own 'Z' message after completion, so ideally
the above code should receive the message only for worker which has
sent the message. I think for that it needs worker information who has
sent the message.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Thu, Jan 15, 2015 at 7:00 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jan 13, 2015 at 1:33 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Jan 8, 2015 at 6:52 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:+ seg = dsm_attach(DatumGetInt32(main_arg));
Here, I think DatumGetUInt32() needs to be used instead of
DatumGetInt32() as the segment handle is uint32.OK, I'll change that in the next version.
No issues, I have another question related to below code:
+HandleParallelMessages(void) +{ .. .. + for (i = 0; i < pcxt->nworkers; ++i) + { + /* + * Read messages for as long as we have an error queue; if we + * have hit (or hit while reading) ReadyForQuery, this will go to + * NULL. + */ + while (pcxt->worker[i].error_mqh != NULL) + { + shm_mq_result res; + + CHECK_FOR_INTERRUPTS(); + + res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes, + &data, true); + if (res == SHM_MQ_SUCCESS)Here we are checking the error queue for all the workers and this loop
will continue untill all have sent ReadyForQuery() message ('Z') which
will make this loop continue till all workers have finished their work.
Assume situation where first worker has completed the work and sent
'Z' message and second worker is still sending some tuples, now above
code will keep on waiting for 'Z' message from second worker and won't
allow to receive tuples sent by second worker till it send 'Z' message.As each worker send its own 'Z' message after completion, so ideally
the above code should receive the message only for worker which has
sent the message. I think for that it needs worker information who has
sent the message.
Are you talking about HandleParallelMessages() or
WaitForParallelWorkersToFinish()? The former doesn't wait for
anything; it just handles any messages that are available now. The
latter does wait for all workers to finish, but the intention is that
you only call it when you're ready to wind up the entire parallel
operation, so that's OK.
--
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:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Jan 15, 2015 at 7:00 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
+HandleParallelMessages(void) +{ .. .. + for (i = 0; i < pcxt->nworkers; ++i) + { + /* + * Read messages for as long as we have an error queue; if we + * have hit (or hit while reading) ReadyForQuery, this will go to + * NULL. + */ + while (pcxt->worker[i].error_mqh != NULL) + { + shm_mq_result res; + + CHECK_FOR_INTERRUPTS(); + + res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes, + &data, true); + if (res == SHM_MQ_SUCCESS)Here we are checking the error queue for all the workers and this loop
will continue untill all have sent ReadyForQuery() message ('Z') which
will make this loop continue till all workers have finished their work.
Assume situation where first worker has completed the work and sent
'Z' message and second worker is still sending some tuples, now above
code will keep on waiting for 'Z' message from second worker and won't
allow to receive tuples sent by second worker till it send 'Z' message.As each worker send its own 'Z' message after completion, so ideally
the above code should receive the message only for worker which has
sent the message. I think for that it needs worker information who has
sent the message.Are you talking about HandleParallelMessages() or
WaitForParallelWorkersToFinish()? The former doesn't wait for
anything; it just handles any messages that are available now.
I am talking about HandleParallelMessages(). It doesn't wait but
it is looping which will make it run for longer time as explained
above. Just imagine a case where there are two workers and first
worker has sent 'Z' message and second worker is doing some
work, now in such a scenario loop will not finish until second worker
also send 'Z' message or error. Am I missing something?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Thu, Jan 15, 2015 at 9:09 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Thu, Jan 15, 2015 at 6:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Thu, Jan 15, 2015 at 7:00 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:+HandleParallelMessages(void) +{ .. .. + for (i = 0; i < pcxt->nworkers; ++i) + { + /* + * Read messages for as long as we have an error queue; if we + * have hit (or hit while reading) ReadyForQuery, this will go to + * NULL. + */ + while (pcxt->worker[i].error_mqh != NULL) + { + shm_mq_result res; + + CHECK_FOR_INTERRUPTS(); + + res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes, + &data, true); + if (res == SHM_MQ_SUCCESS)Here we are checking the error queue for all the workers and this loop
will continue untill all have sent ReadyForQuery() message ('Z') which
will make this loop continue till all workers have finished their work.
Assume situation where first worker has completed the work and sent
'Z' message and second worker is still sending some tuples, now above
code will keep on waiting for 'Z' message from second worker and won't
allow to receive tuples sent by second worker till it send 'Z' message.As each worker send its own 'Z' message after completion, so ideally
the above code should receive the message only for worker which has
sent the message. I think for that it needs worker information who has
sent the message.Are you talking about HandleParallelMessages() or
WaitForParallelWorkersToFinish()? The former doesn't wait for
anything; it just handles any messages that are available now.I am talking about HandleParallelMessages(). It doesn't wait but
it is looping which will make it run for longer time as explained
above. Just imagine a case where there are two workers and first
worker has sent 'Z' message and second worker is doing some
work, now in such a scenario loop will not finish until second worker
also send 'Z' message or error. Am I missing something?
Blah. You're right. I intended to write this loop so that it only
runs until shm_mq_receive() returns SHM_MQ_WOULD_BLOCK. But that's
not what I did. Will fix.
--
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-01-05 11:27:57 -0500, Robert Haas wrote:
On Sat, Jan 3, 2015 at 7:31 PM, Andres Freund <andres@2ndquadrant.com> wrote:
* I wonder if parallel contexts shouldn't be tracked via resowners
That is a good question. I confess that I'm somewhat fuzzy about
which things should be tracked via the resowner mechanism vs. which
things should have their own bespoke bookkeeping. However, the
AtEOXact_Parallel() stuff happens well before ResourceOwnerRelease(),
which makes merging them seem not particularly clean.
I'm not sure either. But I think the current location is wrong anyway -
during AtEOXact_Parallel() before running user defined queries via
AfterTriggerFireDeferred() seems wrong.
* combocid.c should error out in parallel mode, as insurance
Eh, which function? HeapTupleHeaderGetCmin(),
HeapTupleHeaderGetCmax(), and AtEOXact_ComboCid() are intended to work
in parallel mode. HeapTupleHeaderAdjustCmax() could
Assert(!IsInParallelMode()) but the only calls are in heap_update()
and heap_delete() which already have error checks, so putting yet
another elog() there seems like overkill.
To me it seems like a good idea, but whatever.
* I doubt it's a good idea to allow heap_insert, heap_inplace_update,
index_insert. I'm not convinced that those will be handled correct and
relaxing restrictions is easier than adding them.I'm fine with adding checks to heap_insert() and
heap_inplace_update(). I'm not sure we really need to add anything to
index_insert(); how are we going to get there without hitting some
other prohibited operation first?
I'm not sure. But it's not that hard to imagine that somebody will start
adding codepaths that insert into indexes first. Think upsert.
* I'd much rather separate HandleParallelMessageInterrupt() in one
variant that just tells the machinery there's a interrupt (called from
signal handlers) and one that actually handles them. We shouldn't even
consider adding any new code that does allocations, errors and such in
signal handlers. That's just a *bad* idea.That's a nice idea, but I didn't invent the way this crap works today.
ImmediateInterruptOK gets set to true while performing a lock wait,
and we need to be able to respond to errors while in that state. I
think there's got to be a better way to handle that than force every
asynchronous operation to have to cope with the fact that
ImmediateInterruptOK may be set or not set, but as of today that's
what you have to do.
I personally think it's absolutely unacceptable to add any more of
that. That the current mess works is more luck than anything else - and
I'm pretty sure there's several bugs in it. But since I realize I can't
force you to develop a alternative solution, I tried to implement enough
infrastructure to deal with it without too much work.
As far as I can see this could relatively easily be implemented ontop of
the removal of ImmediateInterruptOK in the patch series I posted
yesterday? Afaics you just need to remove most of
+HandleParallelMessageInterrupt() and replace it by a single SetLatch().
* I'm not a fan of the shm_toc stuff... Verbose and hard to read. I'd
much rather place a struct there and be careful about not using
pointers. That also would obliviate the need to reserve some ids.I don't see how that's going to work with variable-size data
structures, and a bunch of the things that we need to serialize are
variable-size.
Meh. Just appending the variable data to the end of the structure and
calculating offsets works just fine.
Moreover, I'm not really interested in rehashing this
design again. I know it's not your favorite; you've said that before.
Well, if I keep having to read code using it, I'll keep being annoyed by
it. I guess we both have to live with that.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Fri, Jan 16, 2015 at 8:05 AM, Andres Freund <andres@2ndquadrant.com> wrote:
I'm not sure either. But I think the current location is wrong anyway -
during AtEOXact_Parallel() before running user defined queries via
AfterTriggerFireDeferred() seems wrong.
Good point.
I'm fine with adding checks to heap_insert() and
heap_inplace_update(). I'm not sure we really need to add anything to
index_insert(); how are we going to get there without hitting some
other prohibited operation first?I'm not sure. But it's not that hard to imagine that somebody will start
adding codepaths that insert into indexes first. Think upsert.
OK, but what's the specific reason it's unsafe? The motivation for
prohibiting update and delete is that, if a new combo CID were to be
created mid-scan, we have no way to make other workers aware of it.
There's no special reason to think that heap_insert() or
heap_inplace_update() are unsafe, but, sure, we can prohibit them for
symmetry. If we're going to start extending the net further and
further, we should have specific comments explaining what the hazards.
People will eventually want to relax these restrictions, I think, and
there's nothing scarier than removing a prohibition that has
absolutely no comments to explain why that thing was restricted in the
first place.
As far as I can see this could relatively easily be implemented ontop of
the removal of ImmediateInterruptOK in the patch series I posted
yesterday? Afaics you just need to remove most of
+HandleParallelMessageInterrupt() and replace it by a single SetLatch().
That would be swell. I'll have a look, when I have time, or when it's
committed, whichever happens first.
* I'm not a fan of the shm_toc stuff... Verbose and hard to read. I'd
much rather place a struct there and be careful about not using
pointers. That also would obliviate the need to reserve some ids.I don't see how that's going to work with variable-size data
structures, and a bunch of the things that we need to serialize are
variable-size.Meh. Just appending the variable data to the end of the structure and
calculating offsets works just fine.
I think coding it all up ad-hoc would be pretty bug-prone. This
provides some structure, where each module only needs to know about
its own serialization format. To me, that's a lot easier to work
with.
New patch attached. I'm going to take the risk of calling this v1
(previous versions have been 0.x), since I've now done something about
the heavyweight locking issue, as well as fixed the message-looping
bug Amit pointed out. It doubtless needs more work, but it's starting
to smell a bit more like a real patch.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v1.patchtext/x-patch; charset=US-ASCII; name=parallel-mode-v1.patchDownload
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..3c0ae7d
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,12 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_sleep(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
+
+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_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..0a32ea8
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,238 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.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/snapmgr.h"
+#include "utils/tqual.h"
+
+PG_MODULE_MAGIC;
+
+PG_FUNCTION_INFO_V1(parallel_sleep);
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct
+{
+ int32 sleep_time;
+} ParallelSleepInfo;
+
+typedef struct
+{
+ int32 relid;
+ slock_t mutex;
+ BlockNumber lastblock;
+ BlockNumber currentblock;
+ int64 ntuples;
+} ParallelCountInfo;
+
+void _PG_init(void);
+void sleep_worker_main(dsm_segment *seg, shm_toc *toc);
+void count_worker_main(dsm_segment *seg, shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelCountInfo *info);
+
+Datum
+parallel_sleep(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelSleepInfo *info;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy",
+ "sleep_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelSleepInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelSleepInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ WaitForParallelWorkersToFinish(pcxt);
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelCountInfo *info;
+ Relation rel;
+ int64 result;
+
+ 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_dummy",
+ "count_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelCountInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelCountInfo));
+ info->relid = relid;
+ SpinLockInit(&info->mutex);
+ info->lastblock = RelationGetNumberOfBlocks(rel);
+ info->currentblock = 0;
+ info->ntuples = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, info);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ result = info->ntuples;
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(result);
+}
+
+void
+sleep_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelSleepInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
+
+void
+count_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelCountInfo *info;
+ Relation rel;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ rel = relation_open(info->relid, AccessShareLock);
+ count_helper(rel, info);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelCountInfo *info)
+{
+ int64 ntuples = 0;
+ int64 mytuples = 0;
+ Oid relid = info->relid;
+ Snapshot snapshot = GetActiveSnapshot();
+
+ for (;;)
+ {
+ 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);
+ lines = PageGetMaxOffsetNumber(page);
+ ntuples = 0;
+
+ all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
+
+ for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
+ lineoff <= lines;
+ lineoff++, lpp++)
+ {
+ HeapTupleData loctup;
+
+ if (!ItemIdIsNormal(lpp))
+ continue;
+ if (all_visible)
+ {
+ ++ntuples;
+ continue;
+ }
+
+ loctup.t_tableOid = relid;
+ loctup.t_data = (HeapTupleHeader) PageGetItem(page, lpp);
+ loctup.t_len = ItemIdGetLength(lpp);
+
+ if (HeapTupleSatisfiesVisibility(&loctup, snapshot, buffer))
+ ++ntuples;
+ }
+
+ UnlockReleaseBuffer(buffer);
+ }
+
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 21e9d06..57408d3 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..401cfd6
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,206 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..9c67291
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,940 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LOCK UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0009)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 7);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler)
+ && InterruptHoldoffCount == 0 && CritSectionCount == 0
+ && !HandlingParallelMessages)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+ HandlingParallelMessages = true;
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ HandlingParallelMessages = false;
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a ReadyForQuery from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback *save_error_context_stack;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ error_context_stack = pcxt->error_context_stack;
+ ThrowErrorData(&edata);
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..bd6b386 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,32 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only a single TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +139,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +181,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +212,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +382,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +397,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +491,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +549,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +682,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +785,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +868,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +928,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1839,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1970,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2006,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2016,10 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2059,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2092,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2141,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2167,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2193,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2439,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2460,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2509,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2374,6 +2534,13 @@ AbortTransaction(void)
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* do abort processing
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
@@ -2385,9 +2552,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2577,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2601,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2654,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2711,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2747,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3035,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3425,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3603,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3706,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3745,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3777,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3812,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3847,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3925,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3961,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4050,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4086,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4119,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4153,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4172,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4248,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4340,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4444,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4604,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4751,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4799,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4996,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index 0819e80..bc66eac 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -467,7 +467,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 0e604b7..207cfb4 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -914,9 +914,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index fcc42fa..584f83c 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -135,8 +135,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -679,18 +691,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -701,6 +718,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 02f871c..6492067 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -450,7 +450,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1620,7 +1620,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index fe6316e..47ed84c 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5311,7 +5311,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..4945f59 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 61c8d21..112ba69 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -669,7 +669,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ false);
}
/*
@@ -680,13 +681,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * parallelReacquire should be false except for the case of a parallel
+ * worker reacquiring locks already held by the parallel group leader. In
+ * that case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ bool parallelReacquire)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -797,7 +805,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && !parallelReacquire)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
@@ -910,9 +918,12 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
- if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
+ if (parallelReacquire)
+ status = STATUS_OK;
+ else if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
else
status = LockCheckConflicts(lockMethodTable, lockmode,
@@ -3553,6 +3564,84 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, false, true, true);
+ if (result != LOCKACQUIRE_OK)
+ elog(ERROR, "parallel worker lock not reacquired OK");
+ }
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 8f74353..bbad0dc 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2963,7 +2964,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
@@ -3676,7 +3678,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..21d9f73d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -420,6 +420,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +438,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +455,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +473,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +493,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +514,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +534,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +555,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +575,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +595,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +613,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +632,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +650,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +669,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +690,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +712,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +733,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +755,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +776,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +797,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 4646e09..1dc3153 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -453,11 +453,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -468,7 +467,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 1f5cf06..983b237 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index f6df077..d9bfa25 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5656,6 +5656,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6929,6 +6943,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index b4483c5..ab77c8e 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -155,6 +155,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -186,6 +202,10 @@ GetTransactionSnapshot(void)
Assert(RegisteredSnapshots == 0);
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -237,6 +257,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -345,7 +373,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -392,7 +421,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -548,11 +585,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1247,7 +1297,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1350,3 +1400,159 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..761ba1f
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,61 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 6e33a17..ed6eda2 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -259,6 +259,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
@@ -294,7 +295,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -398,7 +399,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index 0460653..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
@@ -130,6 +132,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..5b61ce4 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ bool parallelReacquire);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,9 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(char *start_address);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..5167e17 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On Sat, Jan 17, 2015 at 3:40 AM, Robert Haas <robertmhaas@gmail.com> wrote:
New patch attached. I'm going to take the risk of calling this v1
(previous versions have been 0.x), since I've now done something about
the heavyweight locking issue, as well as fixed the message-looping
bug Amit pointed out. It doubtless needs more work, but it's starting
to smell a bit more like a real patch.
I need some clarification regarding below code:
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus
status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+
save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+
for (;;)
+ {
+ pid_t pid;
+
+
CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+
if (status == BGWH_STOPPED)
+ return status;
+
+ rc =
WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET |
WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+
return BGWH_POSTMASTER_DIED;
It seems this code has possibility to wait forever.
Assume one of the worker is not able to start (not able to attach
to shared memory or some other reason), then status returned by
GetBackgroundWorkerPid() will be BGWH_NOT_YET_STARTED
and after that it can wait forever in WaitLatch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Tue, Jan 20, 2015 at 9:41 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
It seems [WaitForBackgroundWorkerShutdown] has possibility to wait forever.
Assume one of the worker is not able to start (not able to attach
to shared memory or some other reason), then status returned by
GetBackgroundWorkerPid() will be BGWH_NOT_YET_STARTED
and after that it can wait forever in WaitLatch.
I don't think that's right. The status only remains
BGWH_NOT_YET_STARTED until the postmaster forks the child process. At
that point it immediately changes to BGWH_STARTED. If it starts up
and then dies early on, for example because it can't attach to shared
memory or somesuch, the status will change to BGWH_STOPPED.
--
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 20, 2015 at 9:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Jan 20, 2015 at 9:41 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
It seems [WaitForBackgroundWorkerShutdown] has possibility to wait
forever.
Assume one of the worker is not able to start (not able to attach
to shared memory or some other reason), then status returned by
GetBackgroundWorkerPid() will be BGWH_NOT_YET_STARTED
and after that it can wait forever in WaitLatch.I don't think that's right. The status only remains
BGWH_NOT_YET_STARTED until the postmaster forks the child process.
I think the control flow can reach the above location before
postmaster could fork all the workers. Consider a case that
we have launched all workers during ExecutorStart phase
and then before postmaster could start all workers an error
occurs in master backend and then it try to Abort the transaction
and destroy the parallel context, at that moment it will get the
above status and wait forever in above code.
I am able to reproduce above scenario with debugger by using
parallel_seqscan patch.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Jan 21, 2015 at 2:11 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
On Tue, Jan 20, 2015 at 9:52 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Tue, Jan 20, 2015 at 9:41 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:It seems [WaitForBackgroundWorkerShutdown] has possibility to wait
forever.
Assume one of the worker is not able to start (not able to attach
to shared memory or some other reason), then status returned by
GetBackgroundWorkerPid() will be BGWH_NOT_YET_STARTED
and after that it can wait forever in WaitLatch.I don't think that's right. The status only remains
BGWH_NOT_YET_STARTED until the postmaster forks the child process.I think the control flow can reach the above location before
postmaster could fork all the workers. Consider a case that
we have launched all workers during ExecutorStart phase
and then before postmaster could start all workers an error
occurs in master backend and then it try to Abort the transaction
and destroy the parallel context, at that moment it will get the
above status and wait forever in above code.I am able to reproduce above scenario with debugger by using
parallel_seqscan patch.
Hmm. Well, if you can reproduce it, there clearly must be a bug. But
I'm not quite sure where. What should happen in that case is that the
process that started the worker has to wait for the postmaster to
actually start it, and then after that for the new process to die, and
then it should return.
--
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, Jan 21, 2015 at 6:35 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Wed, Jan 21, 2015 at 2:11 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:
On Tue, Jan 20, 2015 at 9:52 PM, Robert Haas <robertmhaas@gmail.com>
wrote:
On Tue, Jan 20, 2015 at 9:41 AM, Amit Kapila <amit.kapila16@gmail.com>
wrote:It seems [WaitForBackgroundWorkerShutdown] has possibility to wait
forever.
Assume one of the worker is not able to start (not able to attach
to shared memory or some other reason), then status returned by
GetBackgroundWorkerPid() will be BGWH_NOT_YET_STARTED
and after that it can wait forever in WaitLatch.I don't think that's right. The status only remains
BGWH_NOT_YET_STARTED until the postmaster forks the child process.I think the control flow can reach the above location before
postmaster could fork all the workers. Consider a case that
we have launched all workers during ExecutorStart phase
and then before postmaster could start all workers an error
occurs in master backend and then it try to Abort the transaction
and destroy the parallel context, at that moment it will get the
above status and wait forever in above code.I am able to reproduce above scenario with debugger by using
parallel_seqscan patch.Hmm. Well, if you can reproduce it, there clearly must be a bug. But
I'm not quite sure where. What should happen in that case is that the
process that started the worker has to wait for the postmaster to
actually start it,
Okay, I think this should solve the issue, also it should be done
before call of TerminateBackgroundWorker().
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Jan 16, 2015 at 5:10 PM, Robert Haas <robertmhaas@gmail.com> wrote:
New patch attached. I'm going to take the risk of calling this v1
(previous versions have been 0.x), since I've now done something about
the heavyweight locking issue, as well as fixed the message-looping
bug Amit pointed out. It doubtless needs more work, but it's starting
to smell a bit more like a real patch.
Here's a new version. Andres mentioned previously that he thought it
would be a good idea to commit the addition of
BackgroundWorkerInitializeConnectionByOid() separately, as he's had
cause to want it independently of the rest of this stuff. It would be
useful for pg_background, too. So I've broken that out into a
separate patch here (bgworker-by-oid.patch) and will commit that RSN
unless somebody thinks it's a bad idea for some reason. AFAICS it
should be uncontroversial.
The main patch (parallel-mode-v2.patch) has evolved just a bit more
since the previous version. It now arranges for all libraries
libraries which we'd dynamically loaded into the original process to
be loaded into the worker as well, which fixes a possible failure when
those libraries define custom GUCs that need to be properly restored.
I'm wondering how much more there really is to do here. From the
parallel sequential scan discussion, and off-list conversations with
Amit, it's becoming clear to me that there are a bunch of things that
are generic to query planning and execution that this patch doesn't
currently consider, so Amit's handling them in the parallel sequential
scan patch. It seems likely that some of that stuff should be pulled
out of that patch and formed into some more generic infrastructure.
But I don't favor putting that stuff in this patch, because there are
already useful things you can do with what I've got here. If you want
to write a parallel version of some SQL-callable function, for
example, you can do that without anything more than this. You could
probably also parallelize utility commands with just this
infrastructure. The stuff that may need to be pulled out of Amit's
patch is generic to query planning and execution, but probably not for
other applications of server-side parallelism. That seems like a
relatively coherent place to draw a line. So I'm feeling like it
might be just as well to push this much into the tree and move on to
the next set of problems. Of course, I don't want to preempt anyone's
desire to review and comment on this further, either.
The final patch attached her (parallel-dummy-v2.patch) has been
updated slightly to incorporate some prefetching logic. It's still
just demo code and is not intended for commit. I'm not sure whether
the prefetching logic can actually be made to improve performance,
either; if anyone feels like playing with that and reporting results
back, that would be swell.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
bgworker-by-oid.patchbinary/octet-stream; name=bgworker-by-oid.patchDownload
commit a2f38ec05fcbd1fc7f97399806b1edaa6688c7f1
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 09:55:06 2015 -0500
Allow background workers to connect using OIDs rather than names.
If the username and database name are being taken from a string,
perhaps a GUC, the existing BackgroundWorkerInitializeConnection()
interface works well; however, in some cases, we may want a worker
to connect to the same database as the same user as some existing
session. In such cases, it's better to specify the connection
information using OIDs. This patch adds a new function called
BackgroundWorkerInitializeConnectionByOid for that purpose.
Patch by me, reviewed by Andres Freund.
diff --git a/doc/src/sgml/bgworker.sgml b/doc/src/sgml/bgworker.sgml
index 8e218ac..ef28f72 100644
--- a/doc/src/sgml/bgworker.sgml
+++ b/doc/src/sgml/bgworker.sgml
@@ -146,14 +146,17 @@ typedef struct BackgroundWorker
</para>
<para>Once running, the process can connect to a database by calling
- <function>BackgroundWorkerInitializeConnection(<parameter>char *dbname</parameter>, <parameter>char *username</parameter>)</function>.
+ <function>BackgroundWorkerInitializeConnection(<parameter>char *dbname</parameter>, <parameter>char *username</parameter>)</function> or
+ <function>BackgroundWorkerInitializeConnectionByOid(<parameter>Oid dboid</parameter>, <parameter>Oid useroid</parameter>)</function>.
This allows the process to run transactions and queries using the
- <literal>SPI</literal> interface. If <varname>dbname</> is NULL,
- the session is not connected to any particular database, but shared catalogs
- can be accessed. If <varname>username</> is NULL, the process will run as
- the superuser created during <command>initdb</>.
- BackgroundWorkerInitializeConnection can only be called once per background
- process, it is not possible to switch databases.
+ <literal>SPI</literal> interface. If <varname>dbname</> is NULL or
+ <varname>dboid</> is <literal>InvalidOid</>, the session is not connected
+ to any particular database, but shared catalogs can be accessed.
+ If <varname>username</> is NULL or <varname>useroid</> is
+ <literal>InvalidOid</>, the process will run as the superuser created
+ during <command>initdb</>.
+ A background worker can only call one of these two functions, and only
+ once. It is not possible to switch databases.
</para>
<para>
diff --git a/src/backend/bootstrap/bootstrap.c b/src/backend/bootstrap/bootstrap.c
index 0819e80..bc66eac 100644
--- a/src/backend/bootstrap/bootstrap.c
+++ b/src/backend/bootstrap/bootstrap.c
@@ -467,7 +467,7 @@ BootstrapModeMain(void)
*/
InitProcess();
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
/* Initialize stuff for bootstrap-file processing */
for (i = 0; i < MAXATTR; i++)
diff --git a/src/backend/postmaster/autovacuum.c b/src/backend/postmaster/autovacuum.c
index 02f871c..6492067 100644
--- a/src/backend/postmaster/autovacuum.c
+++ b/src/backend/postmaster/autovacuum.c
@@ -450,7 +450,7 @@ AutoVacLauncherMain(int argc, char *argv[])
InitProcess();
#endif
- InitPostgres(NULL, InvalidOid, NULL, NULL);
+ InitPostgres(NULL, InvalidOid, NULL, InvalidOid, NULL);
SetProcessingMode(NormalProcessing);
@@ -1620,7 +1620,7 @@ AutoVacWorkerMain(int argc, char *argv[])
* Note: if we have selected a just-deleted database (due to using
* stale stats info), we'll fail and exit here.
*/
- InitPostgres(NULL, dbid, NULL, dbname);
+ InitPostgres(NULL, dbid, NULL, InvalidOid, dbname);
SetProcessingMode(NormalProcessing);
set_ps_display(dbname, false);
ereport(DEBUG1,
diff --git a/src/backend/postmaster/postmaster.c b/src/backend/postmaster/postmaster.c
index fe6316e..47ed84c 100644
--- a/src/backend/postmaster/postmaster.c
+++ b/src/backend/postmaster/postmaster.c
@@ -5311,7 +5311,30 @@ BackgroundWorkerInitializeConnection(char *dbname, char *username)
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("database connection requirement not indicated during registration")));
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
+
+ /* it had better not gotten out of "init" mode yet */
+ if (!IsInitProcessingMode())
+ ereport(ERROR,
+ (errmsg("invalid processing mode in background worker")));
+ SetProcessingMode(NormalProcessing);
+}
+
+/*
+ * Connect background worker to a database using OIDs.
+ */
+void
+BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid)
+{
+ BackgroundWorker *worker = MyBgworkerEntry;
+
+ /* XXX is this the right errcode? */
+ if (!(worker->bgw_flags & BGWORKER_BACKEND_DATABASE_CONNECTION))
+ ereport(FATAL,
+ (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+ errmsg("database connection requirement not indicated during registration")));
+
+ InitPostgres(NULL, dboid, NULL, useroid, NULL);
/* it had better not gotten out of "init" mode yet */
if (!IsInitProcessingMode())
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 8f74353..0ce637a 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -3676,7 +3676,7 @@ PostgresMain(int argc, char *argv[],
* it inside InitPostgres() instead. In particular, anything that
* involves database access should be there, not here.
*/
- InitPostgres(dbname, InvalidOid, username, NULL);
+ InitPostgres(dbname, InvalidOid, username, InvalidOid, NULL);
/*
* If the PostmasterContext is still around, recycle the space; we don't
diff --git a/src/backend/utils/init/miscinit.c b/src/backend/utils/init/miscinit.c
index 4646e09..1dc3153 100644
--- a/src/backend/utils/init/miscinit.c
+++ b/src/backend/utils/init/miscinit.c
@@ -453,11 +453,10 @@ has_rolreplication(Oid roleid)
* Initialize user identity during normal backend startup
*/
void
-InitializeSessionUserId(const char *rolename)
+InitializeSessionUserId(const char *rolename, Oid roleid)
{
HeapTuple roleTup;
Form_pg_authid rform;
- Oid roleid;
/*
* Don't do scans if we're bootstrapping, none of the system catalogs
@@ -468,7 +467,10 @@ InitializeSessionUserId(const char *rolename)
/* call only once */
AssertState(!OidIsValid(AuthenticatedUserId));
- roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ if (rolename != NULL)
+ roleTup = SearchSysCache1(AUTHNAME, PointerGetDatum(rolename));
+ else
+ roleTup = SearchSysCache1(AUTHOID, ObjectIdGetDatum(roleid));
if (!HeapTupleIsValid(roleTup))
ereport(FATAL,
(errcode(ERRCODE_INVALID_AUTHORIZATION_SPECIFICATION),
diff --git a/src/backend/utils/init/postinit.c b/src/backend/utils/init/postinit.c
index 1f5cf06..983b237 100644
--- a/src/backend/utils/init/postinit.c
+++ b/src/backend/utils/init/postinit.c
@@ -523,6 +523,9 @@ BaseInit(void)
* name can be returned to the caller in out_dbname. If out_dbname isn't
* NULL, it must point to a buffer of size NAMEDATALEN.
*
+ * Similarly, the username can be passed by name, using the username parameter,
+ * or by OID using the useroid parameter.
+ *
* In bootstrap mode no parameters are used. The autovacuum launcher process
* doesn't use any parameters either, because it only goes far enough to be
* able to read pg_database; it doesn't connect to any particular database.
@@ -537,7 +540,7 @@ BaseInit(void)
*/
void
InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname)
+ Oid useroid, char *out_dbname)
{
bool bootstrap = IsBootstrapProcessingMode();
bool am_superuser;
@@ -692,18 +695,18 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
(errcode(ERRCODE_UNDEFINED_OBJECT),
errmsg("no roles are defined in this database system"),
errhint("You should immediately run CREATE USER \"%s\" SUPERUSER;.",
- username)));
+ username != NULL ? username : "postgres")));
}
else if (IsBackgroundWorker)
{
- if (username == NULL)
+ if (username == NULL && !OidIsValid(useroid))
{
InitializeSessionUserIdStandalone();
am_superuser = true;
}
else
{
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
}
@@ -712,7 +715,7 @@ InitPostgres(const char *in_dbname, Oid dboid, const char *username,
/* normal multiuser case */
Assert(MyProcPort != NULL);
PerformAuthentication(MyProcPort);
- InitializeSessionUserId(username);
+ InitializeSessionUserId(username, useroid);
am_superuser = superuser();
}
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 6e33a17..83198ed 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -294,7 +294,7 @@ extern bool InLocalUserIdChange(void);
extern bool InSecurityRestrictedOperation(void);
extern void GetUserIdAndContext(Oid *userid, bool *sec_def_context);
extern void SetUserIdAndContext(Oid userid, bool sec_def_context);
-extern void InitializeSessionUserId(const char *rolename);
+extern void InitializeSessionUserId(const char *rolename, Oid useroid);
extern void InitializeSessionUserIdStandalone(void);
extern void SetSessionAuthorization(Oid userid, bool is_superuser);
extern Oid GetCurrentRoleId(void);
@@ -398,7 +398,7 @@ extern AuxProcType MyAuxProcType;
extern void pg_split_opts(char **argv, int *argcp, char *optstr);
extern void InitializeMaxBackends(void);
extern void InitPostgres(const char *in_dbname, Oid dboid, const char *username,
- char *out_dbname);
+ Oid useroid, char *out_dbname);
extern void BaseInit(void);
/* in utils/init/miscinit.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index 0460653..a81b90b 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -130,6 +130,9 @@ extern PGDLLIMPORT BackgroundWorker *MyBgworkerEntry;
*/
extern void BackgroundWorkerInitializeConnection(char *dbname, char *username);
+/* Just like the above, but specifying database and user by OID. */
+extern void BackgroundWorkerInitializeConnectionByOid(Oid dboid, Oid useroid);
+
/* Block/unblock signals in a background worker process */
extern void BackgroundWorkerBlockSignals(void);
extern void BackgroundWorkerUnblockSignals(void);
parallel-mode-v2.patchbinary/octet-stream; name=parallel-mode-v2.patchDownload
commit 4ad1db0a6c907a56a9aa1ffbce1a1d8e84df80e5
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 08:39:21 2015 -0500
Create an infrastructure for parallel computation in PostgreSQL.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Heikki Linnakangas, Jim Nasby, Simon Riggs,
and Andres Freund.
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 21e9d06..57408d3 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..bcdb18b
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,208 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..1a66acc
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,960 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessages(void);
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(bool signal_handler)
+{
+ int save_errno = errno;
+
+ /* Don't joggle the elbow of proc_exit */
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ /*
+ * If it's safe to interrupt, service the interrupt immediately.
+ * (We shouldn't be in parallel mode if waiting for the user to send
+ * a new query, but we could be waiting for a lock.)
+ */
+ if ((ImmediateInterruptOK || !signal_handler)
+ && InterruptHoldoffCount == 0 && CritSectionCount == 0
+ && !HandlingParallelMessages)
+ {
+ bool notify_enabled;
+ bool catchup_enabled;
+ bool save_ImmediateInterruptOK;
+
+ /*
+ * Disable everything that might recursively interrupt us.
+ *
+ * If there were any possibility that disabling and re-enabling
+ * interrupts or handling parallel messages might take a lock, we'd
+ * need to HOLD_INTERRUPTS() as well, since taking a lock might
+ * cause ImmediateInterruptOK to get temporarily reset to true.
+ * But that shouldn't happen, so this is (hopefully) safe. That's
+ * good, because it lets us respond to query cancel and die
+ * interrupts while we're in the midst of message-processing.
+ */
+ save_ImmediateInterruptOK = ImmediateInterruptOK;
+ ImmediateInterruptOK = false;
+ notify_enabled = DisableNotifyInterrupt();
+ catchup_enabled = DisableCatchupInterrupt();
+ HandlingParallelMessages = true;
+
+ /* OK, do the work... */
+ HandleParallelMessages();
+
+ /* Now re-enable whatever was enabled before */
+ HandlingParallelMessages = false;
+ if (catchup_enabled)
+ EnableCatchupInterrupt();
+ if (notify_enabled)
+ EnableNotifyInterrupt();
+ ImmediateInterruptOK = save_ImmediateInterruptOK;
+ }
+ }
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+static void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a ReadyForQuery from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ CHECK_FOR_INTERRUPTS();
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback *save_error_context_stack;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ error_context_stack = pcxt->error_context_stack;
+ ThrowErrorData(&edata);
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..bd6b386 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,32 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only a single TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +139,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +181,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +212,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +382,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +397,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +491,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +549,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +682,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +785,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +868,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +928,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1839,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1970,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2006,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2016,10 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2059,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2092,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2141,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2167,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2193,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2439,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2460,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2509,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2374,6 +2534,13 @@ AbortTransaction(void)
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* do abort processing
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
@@ -2385,9 +2552,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2577,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2601,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2654,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2711,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2747,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3035,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3425,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3603,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3706,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3745,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3777,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3812,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3847,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3925,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3961,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4050,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4086,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4119,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4153,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4172,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4248,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4340,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4444,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4604,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4751,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4799,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4996,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 8cb2f13..178eea9 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -917,9 +917,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 20b3188..b8222d9 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -713,6 +730,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..4945f59 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt(true);
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..9129fa4 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -669,7 +669,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ false);
}
/*
@@ -680,13 +681,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * parallelReacquire should be false except for the case of a parallel
+ * worker reacquiring locks already held by the parallel group leader. In
+ * that case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ bool parallelReacquire)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -797,7 +805,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && !parallelReacquire)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
@@ -910,9 +918,12 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
- if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
+ if (parallelReacquire)
+ status = STATUS_OK;
+ else if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
else
status = LockCheckConflicts(lockMethodTable, lockmode,
@@ -3557,6 +3568,84 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, false, true, true);
+ if (result != LOCKACQUIRE_OK)
+ elog(ERROR, "parallel worker lock not reacquired OK");
+ }
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 0ce637a..bbad0dc 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2963,7 +2964,8 @@ ProcessInterrupts(void)
errmsg("canceling statement due to user request")));
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessageInterrupt(false);
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..21d9f73d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -420,6 +420,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +438,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +455,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +473,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +493,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +514,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +534,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +555,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +575,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +595,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +613,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +632,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +650,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +669,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +690,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +712,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +733,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +755,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +776,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +797,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 1b69322..affb23a 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -697,3 +698,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9572777..de988ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5602,6 +5602,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6906,6 +6920,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..cb0f412 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,159 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..761ba1f
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,61 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(bool signal_handler);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index 83198ed..ed6eda2 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -259,6 +259,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..5b61ce4 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ bool parallelReacquire);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,9 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(char *start_address);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..5167e17 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
parallel-dummy-v2.patchbinary/octet-stream; name=parallel-dummy-v2.patchDownload
commit b68b38ab1b648676eb989ddd20de2dbdeb39382b
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 08:39:22 2015 -0500
contrib/parallel_dummy
diff --git a/contrib/parallel_dummy/Makefile b/contrib/parallel_dummy/Makefile
new file mode 100644
index 0000000..de00f50
--- /dev/null
+++ b/contrib/parallel_dummy/Makefile
@@ -0,0 +1,19 @@
+MODULE_big = parallel_dummy
+OBJS = parallel_dummy.o $(WIN32RES)
+PGFILEDESC = "parallel_dummy - dummy use of parallel infrastructure"
+
+EXTENSION = parallel_dummy
+DATA = parallel_dummy--1.0.sql
+
+REGRESS = parallel_dummy
+
+ifdef USE_PGXS
+PG_CONFIG = pg_config
+PGXS := $(shell $(PG_CONFIG) --pgxs)
+include $(PGXS)
+else
+subdir = contrib/parallel_dummy
+top_builddir = ../..
+include $(top_builddir)/src/Makefile.global
+include $(top_srcdir)/contrib/contrib-global.mk
+endif
diff --git a/contrib/parallel_dummy/parallel_dummy--1.0.sql b/contrib/parallel_dummy/parallel_dummy--1.0.sql
new file mode 100644
index 0000000..3c0ae7d
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy--1.0.sql
@@ -0,0 +1,12 @@
+-- complain if script is sourced in psql, rather than via CREATE EXTENSION
+\echo Use "CREATE EXTENSION parallel_dummy" to load this file. \quit
+
+CREATE FUNCTION parallel_sleep(sleep_time pg_catalog.int4,
+ nworkers pg_catalog.int4)
+ RETURNS pg_catalog.void STRICT
+ AS 'MODULE_PATHNAME' LANGUAGE C;
+
+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_dummy/parallel_dummy.c b/contrib/parallel_dummy/parallel_dummy.c
new file mode 100644
index 0000000..f92030e
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.c
@@ -0,0 +1,285 @@
+/*--------------------------------------------------------------------------
+ *
+ * parallel_dummy.c
+ * Test harness code for parallel mode code.
+ *
+ * Copyright (C) 2013-2014, PostgreSQL Global Development Group
+ *
+ * IDENTIFICATION
+ * contrib/parallel_dummy/parallel_dummy.c
+ *
+ * -------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/heapam.h"
+#include "access/parallel.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_sleep);
+PG_FUNCTION_INFO_V1(parallel_count);
+
+#define PARALLEL_DUMMY_KEY 1
+
+typedef struct
+{
+ int32 sleep_time;
+} ParallelSleepInfo;
+
+typedef struct
+{
+ int32 relid;
+ slock_t mutex;
+ BlockNumber lastblock;
+ BlockNumber currentblock;
+ BlockNumber prefetchblock;
+ int64 ntuples;
+} ParallelCountInfo;
+
+void _PG_init(void);
+void sleep_worker_main(dsm_segment *seg, shm_toc *toc);
+void count_worker_main(dsm_segment *seg, shm_toc *toc);
+
+static void count_helper(Relation rel, ParallelCountInfo *info);
+
+int prefetch_distance;
+int prefetch_increment;
+
+void
+_PG_init()
+{
+ DefineCustomIntVariable("parallel_dummy.prefetch_distance",
+ "Sets the prefetch distance in blocks.",
+ NULL, &prefetch_distance,
+ 0, 0, INT_MAX, PGC_USERSET, 0, NULL, NULL, NULL);
+ DefineCustomIntVariable("parallel_dummy.prefetch_increment",
+ "Sets the prefetch increment in blocks.",
+ NULL, &prefetch_increment,
+ 8, 1, INT_MAX, PGC_USERSET, 0, NULL, NULL, NULL);
+}
+
+
+Datum
+parallel_sleep(PG_FUNCTION_ARGS)
+{
+ int32 sleep_time = PG_GETARG_INT32(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelSleepInfo *info;
+
+ if (nworkers < 0)
+ ereport(ERROR,
+ (errmsg("number of parallel workers must be non-negative")));
+
+ if (!already_in_parallel_mode)
+ EnterParallelMode();
+
+ pcxt = CreateParallelContextForExtension("parallel_dummy",
+ "sleep_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelSleepInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelSleepInfo));
+ info->sleep_time = sleep_time;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) sleep_time));
+
+ WaitForParallelWorkersToFinish(pcxt);
+ DestroyParallelContext(pcxt);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_VOID();
+}
+
+Datum
+parallel_count(PG_FUNCTION_ARGS)
+{
+ Oid relid = PG_GETARG_OID(0);
+ int32 nworkers = PG_GETARG_INT32(1);
+ bool already_in_parallel_mode = IsInParallelMode();
+ ParallelContext *pcxt;
+ ParallelCountInfo *info;
+ Relation rel;
+ int64 result;
+
+ 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_dummy",
+ "count_worker_main",
+ nworkers);
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(ParallelCountInfo));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ InitializeParallelDSM(pcxt);
+ info = shm_toc_allocate(pcxt->toc, sizeof(ParallelCountInfo));
+ info->relid = relid;
+ SpinLockInit(&info->mutex);
+ info->lastblock = RelationGetNumberOfBlocks(rel);
+ info->currentblock = 0;
+ info->prefetchblock = 0;
+ info->ntuples = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_DUMMY_KEY, info);
+ LaunchParallelWorkers(pcxt);
+
+ /* here's where we do the "real work" ... */
+ count_helper(rel, info);
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ result = info->ntuples;
+
+ DestroyParallelContext(pcxt);
+
+ relation_close(rel, AccessShareLock);
+
+ if (!already_in_parallel_mode)
+ ExitParallelMode();
+
+ PG_RETURN_INT64(result);
+}
+
+void
+sleep_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelSleepInfo *info;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ /* here's where we do the "real work" ... */
+ DirectFunctionCall1(pg_sleep, Float8GetDatum((float8) info->sleep_time));
+}
+
+void
+count_worker_main(dsm_segment *seg, shm_toc *toc)
+{
+ ParallelCountInfo *info;
+ Relation rel;
+
+ info = shm_toc_lookup(toc, PARALLEL_DUMMY_KEY);
+ Assert(info != NULL);
+
+ rel = relation_open(info->relid, AccessShareLock);
+ count_helper(rel, info);
+ relation_close(rel, AccessShareLock);
+}
+
+static void
+count_helper(Relation rel, ParallelCountInfo *info)
+{
+ int64 ntuples = 0;
+ int64 mytuples = 0;
+ Oid relid = info->relid;
+ Snapshot snapshot = GetActiveSnapshot();
+
+ for (;;)
+ {
+ BlockNumber blkno;
+ Buffer buffer;
+ Page page;
+ int lines;
+ OffsetNumber lineoff;
+ ItemId lpp;
+ bool all_visible;
+ bool done = false;
+#ifdef USE_PREFETCH
+ BlockNumber prefetch_blkno = InvalidBlockNumber;
+ uint32 prefetch_count = 0;
+#endif
+
+ CHECK_FOR_INTERRUPTS();
+
+ SpinLockAcquire(&info->mutex);
+ if (info->currentblock >= info->lastblock)
+ done = true;
+ else
+ {
+#ifdef USE_PREFETCH
+ BlockNumber max_prefetch;
+
+ max_prefetch = info->lastblock - info->prefetchblock;
+ if (max_prefetch > 0 &&
+ info->prefetchblock - info->currentblock < prefetch_distance)
+ {
+ prefetch_blkno = info->prefetchblock;
+ prefetch_count = Min(prefetch_increment, max_prefetch);
+ info->prefetchblock += prefetch_count;
+ }
+#endif
+ blkno = info->currentblock++;
+ }
+ info->ntuples += ntuples;
+ SpinLockRelease(&info->mutex);
+
+ mytuples += ntuples;
+ if (done)
+ break;
+
+#ifdef USE_PREFETCH
+ while (prefetch_count > 0)
+ {
+ PrefetchBuffer(rel, MAIN_FORKNUM, prefetch_blkno);
+ ++prefetch_blkno;
+ --prefetch_count;
+ }
+#endif
+
+ buffer = ReadBuffer(rel, blkno);
+ LockBuffer(buffer, BUFFER_LOCK_SHARE);
+ page = BufferGetPage(buffer);
+ lines = PageGetMaxOffsetNumber(page);
+ ntuples = 0;
+
+ all_visible = PageIsAllVisible(page) && !snapshot->takenDuringRecovery;
+
+ for (lineoff = FirstOffsetNumber, lpp = PageGetItemId(page, lineoff);
+ lineoff <= lines;
+ lineoff++, lpp++)
+ {
+ HeapTupleData loctup;
+
+ if (!ItemIdIsNormal(lpp))
+ continue;
+ if (all_visible)
+ {
+ ++ntuples;
+ continue;
+ }
+
+ loctup.t_tableOid = relid;
+ loctup.t_data = (HeapTupleHeader) PageGetItem(page, lpp);
+ loctup.t_len = ItemIdGetLength(lpp);
+
+ if (HeapTupleSatisfiesVisibility(&loctup, snapshot, buffer))
+ ++ntuples;
+ }
+
+ UnlockReleaseBuffer(buffer);
+ }
+
+ elog(NOTICE, "PID %d counted " INT64_FORMAT " tuples", MyProcPid, mytuples);
+}
diff --git a/contrib/parallel_dummy/parallel_dummy.control b/contrib/parallel_dummy/parallel_dummy.control
new file mode 100644
index 0000000..90bae3f
--- /dev/null
+++ b/contrib/parallel_dummy/parallel_dummy.control
@@ -0,0 +1,4 @@
+comment = 'Dummy parallel code'
+default_version = '1.0'
+module_pathname = '$libdir/parallel_dummy'
+relocatable = true
On 1/30/15 11:08 AM, Robert Haas wrote:
The final patch attached her (parallel-dummy-v2.patch) has been
updated slightly to incorporate some prefetching logic. It's still
just demo code and is not intended for commit. I'm not sure whether
the prefetching logic can actually be made to improve performance,
either; if anyone feels like playing with that and reporting results
back, that would be swell.
Wouldn't we want the prefetching to happen after ReadBuffer() instead of
before? This way we risk pushing the buffer we actually want out, plus
if it's not already available the OS probably won't try and read it
until it's read all the prefetch blocks.
--
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 30, 2015 at 6:38 PM, Jim Nasby <Jim.Nasby@bluetreble.com> wrote:
On 1/30/15 11:08 AM, Robert Haas wrote:
The final patch attached her (parallel-dummy-v2.patch) has been
updated slightly to incorporate some prefetching logic. It's still
just demo code and is not intended for commit. I'm not sure whether
the prefetching logic can actually be made to improve performance,
either; if anyone feels like playing with that and reporting results
back, that would be swell.Wouldn't we want the prefetching to happen after ReadBuffer() instead of
before? This way we risk pushing the buffer we actually want out, plus if
it's not already available the OS probably won't try and read it until it's
read all the prefetch blocks.
Please feel free to try it both ways and let me know what you find
out. We've had a decent amount of commentary on this patch and the
parallel sequential scan patch, but AFAICT, not much actual testing.
The best way to find out whether your proposal would be an improvement
is to try it. I have my own theory based on the testing I've done so
far, but what would be better than a theory is some evidence,
preferably collected by diverse users on diverse systems.
--
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 30, 2015 at 12:08 PM, Robert Haas <robertmhaas@gmail.com> wrote:
Here's a new version. Andres mentioned previously that he thought it
would be a good idea to commit the addition of
BackgroundWorkerInitializeConnectionByOid() separately, as he's had
cause to want it independently of the rest of this stuff. It would be
useful for pg_background, too. So I've broken that out into a
separate patch here (bgworker-by-oid.patch) and will commit that RSN
unless somebody thinks it's a bad idea for some reason. AFAICS it
should be uncontroversial.
This is now done.
The main patch needed some updating in light of Andres's recent
assault on ImmediateInterruptOK (final result: Andres 1-0 IIOK) so
here's a new version.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v3.patchapplication/x-patch; name=parallel-mode-v3.patchDownload
commit 08b79c854d527ead12b568bf62374f064282b320
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 08:39:21 2015 -0500
Create an infrastructure for parallel computation in PostgreSQL.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Heikki Linnakangas, Jim Nasby, Simon Riggs,
and Andres Freund.
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 46060bc1..50bede8 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..bcdb18b
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,208 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..60ed40b
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,914 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnaplen, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnaplen, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a ReadyForQuery from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback *save_error_context_stack;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ error_context_stack = pcxt->error_context_stack;
+ ThrowErrorData(&edata);
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..bd6b386 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,32 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only a single TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +139,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +181,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +212,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +382,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +397,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +491,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +549,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +682,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +785,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +868,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +928,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1839,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1970,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2006,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2016,10 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2059,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2092,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2141,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2167,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2193,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2439,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2460,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2509,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2374,6 +2534,13 @@ AbortTransaction(void)
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* do abort processing
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
@@ -2385,9 +2552,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2577,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2601,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2654,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2711,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2747,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3035,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3425,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3603,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3706,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3745,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3777,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3812,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3847,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3925,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3961,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4050,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4086,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4119,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4153,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4172,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4248,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4340,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4444,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4604,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4751,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4799,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4996,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 33b172b..63b65d1 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -713,6 +730,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..9129fa4 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -669,7 +669,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ false);
}
/*
@@ -680,13 +681,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * parallelReacquire should be false except for the case of a parallel
+ * worker reacquiring locks already held by the parallel group leader. In
+ * that case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ bool parallelReacquire)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -797,7 +805,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && !parallelReacquire)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
@@ -910,9 +918,12 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
- if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
+ if (parallelReacquire)
+ status = STATUS_OK;
+ else if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
else
status = LockCheckConflicts(lockMethodTable, lockmode,
@@ -3557,6 +3568,84 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, false, true, true);
+ if (result != LOCKACQUIRE_OK)
+ elog(ERROR, "parallel worker lock not reacquired OK");
+ }
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 28af40c..8899448 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2989,7 +2990,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..21d9f73d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -420,6 +420,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +438,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +455,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +473,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +493,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +514,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +534,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +555,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +575,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +595,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +613,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +632,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +650,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +669,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +690,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +712,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +733,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +755,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +776,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +797,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 1b69322..affb23a 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -697,3 +698,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9572777..de988ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5602,6 +5602,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6906,6 +6920,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..cb0f412 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,159 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, Size maxsize, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ /* If the size is small, throw an error */
+ if (maxsize < EstimateSnapshotSpace(snapshot))
+ elog(ERROR, "not enough space to serialize given snapshot");
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..0b3742f
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,62 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+
+extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers);
+extern ParallelContext *CreateParallelContextForExtension(char *library_name,
+ char *function_name, int nworkers);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..5b61ce4 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ bool parallelReacquire);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,9 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(char *start_address);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..5167e17 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,10 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, Size maxsize,
+ char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
On Wed, Feb 4, 2015 at 9:40 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Jan 30, 2015 at 12:08 PM, Robert Haas <robertmhaas@gmail.com>
wrote:
Here's a new version. Andres mentioned previously that he thought it
would be a good idea to commit the addition of
BackgroundWorkerInitializeConnectionByOid() separately, as he's had
cause to want it independently of the rest of this stuff. It would be
useful for pg_background, too. So I've broken that out into a
separate patch here (bgworker-by-oid.patch) and will commit that RSN
unless somebody thinks it's a bad idea for some reason. AFAICS it
should be uncontroversial.This is now done.
The main patch needed some updating in light of Andres's recent
assault on ImmediateInterruptOK (final result: Andres 1-0 IIOK) so
here's a new version.
I think we should expose variable ParallelWorkerNumber (or if you don't
want to expose it then atleast GetParallel* function call is required to get
the value of same), as that is needed for external applications wherever
they want to allocate something for each worker, some examples w.r.t
parallel seq scan patch are each worker should have separate tuple
queue and probably for implementation of Explain statement also we
might need it.
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Wed, Feb 4, 2015 at 10:47 PM, Amit Kapila <amit.kapila16@gmail.com> wrote:
I think we should expose variable ParallelWorkerNumber (or if you don't
want to expose it then atleast GetParallel* function call is required to get
the value of same), as that is needed for external applications wherever
they want to allocate something for each worker, some examples w.r.t
parallel seq scan patch are each worker should have separate tuple
queue and probably for implementation of Explain statement also we
might need it.
Oh, right. You asked for that before, and I made the variable itself
non-static, but didn't add the prototype to the header file. Oops.
Here's v4, with that fixed and a few more tweaks.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v4.patchapplication/x-patch; name=parallel-mode-v4.patchDownload
commit 8c57754873ece873fb6f5c2d513dbda158b3a487
Author: Robert Haas <rhaas@postgresql.org>
Date: Fri Jan 30 08:39:21 2015 -0500
Create an infrastructure for parallel computation in PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Heikki Linnakangas, Jim Nasby, Simon
Riggs, and Andres Freund.
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 46060bc1..50bede8 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..bcdb18b
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,208 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..e63ec46
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,914 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Are we in the midst of handling parallel messages? */
+static bool HandlingParallelMessages = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelMain(Datum main_arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void handle_sigterm(SIGNAL_ARGS);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExtension(char *library_name, char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit cleanly.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ *
+ * If signal_handler is true, we are being called from a signal handler and must
+ * be extremely cautious about what we do here!
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a ReadyForQuery from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'E':
+ case 'N':
+ {
+ ErrorData edata;
+ ErrorContextCallback *save_error_context_stack;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ error_context_stack = pcxt->error_context_stack;
+ ThrowErrorData(&edata);
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A':
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'Z':
+ {
+ /* ReadyForQuery indicates that this worker exits cleanly. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ HandlingParallelMessages = false;
+
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ ErrorContextCallback errctx;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, handle_sigterm);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /* Install an error-context callback. */
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = NULL;
+ errctx.previous = error_context_stack;
+ error_context_stack = &errctx;
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Handle local_preload_libraries and session_preload_libraries. */
+ process_session_preload_libraries();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ ReadyForQuery(DestRemote);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelMain to be a function
+ * living in a dynamically loaded module, because the module might not be
+ * loaded in every process, or might be loaded but not at the same address.
+ * To work around that problem, CreateParallelContextForExtension() arranges
+ * to call this function rather than calling the extension-provided function
+ * directly; and this function then looks up the real entrypoint and calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * 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
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", MyProcPid);
+}
+
+/*
+ * 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
+handle_sigterm(SIGNAL_ARGS)
+{
+ int save_errno = errno;
+
+ if (MyProc)
+ SetLatch(&MyProc->procLatch);
+
+ if (!proc_exit_inprogress)
+ {
+ InterruptPending = true;
+ ProcDiePending = true;
+ }
+
+ errno = save_errno;
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..bd6b386 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,32 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * Only a single TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +139,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +181,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ bool parallelMode; /* current transaction in parallel operation? */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +212,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ false, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +382,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +397,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +491,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +549,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +682,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(!CurrentTransactionState->parallelMode);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +785,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +868,53 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't let the transaction state be changed
+ * after that point. That includes the parallel mode flag itself.
+ */
+ Assert(!s->parallelMode);
+
+ s->parallelMode = true;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelMode);
+ Assert(!ParallelContextActive());
+
+ s->parallelMode = false;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelMode;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +928,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1839,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1970,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2006,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2016,10 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2059,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2092,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2141,8 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ if (!parallel)
+ AtEOXact_Namespace(true);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2167,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2193,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2284,6 +2439,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2460,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2509,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2374,6 +2534,13 @@ AbortTransaction(void)
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* do abort processing
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
@@ -2385,9 +2552,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2577,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2601,8 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ if (!parallel)
+ AtEOXact_Namespace(false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2654,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelMode = false;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2711,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2747,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3035,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3425,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3603,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3706,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3745,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3777,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3812,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3847,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3925,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3961,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4050,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4086,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4119,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4153,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4172,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4248,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4340,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4444,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4604,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelMode = false;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4751,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelMode = false;
CurrentTransactionState = s;
@@ -4502,6 +4799,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4996,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 33b172b..63b65d1 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -713,6 +730,8 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
if ((rte->requiredPerms & (~ACL_SELECT)) == 0)
continue;
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
+
if (isTempNamespace(get_rel_namespace(rte->relid)))
continue;
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..9129fa4 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -669,7 +669,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ false);
}
/*
@@ -680,13 +681,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * parallelReacquire should be false except for the case of a parallel
+ * worker reacquiring locks already held by the parallel group leader. In
+ * that case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ bool parallelReacquire)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -797,7 +805,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && !parallelReacquire)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
@@ -910,9 +918,12 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
- if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
+ if (parallelReacquire)
+ status = STATUS_OK;
+ else if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
else
status = LockCheckConflicts(lockMethodTable, lockmode,
@@ -3557,6 +3568,84 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, false, true, true);
+ if (result != LOCKACQUIRE_OK)
+ elog(ERROR, "parallel worker lock not reacquired OK");
+ }
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 28af40c..8899448 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2989,7 +2990,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..21d9f73d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -420,6 +420,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +438,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +455,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +473,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +493,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +514,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +534,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +555,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +575,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +595,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +613,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +632,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +650,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +669,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +690,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +712,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +733,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +755,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +776,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +797,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 1b69322..affb23a 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -697,3 +698,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9572777..de988ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5602,6 +5602,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6906,6 +6920,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..9db551d
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,63 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} 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);
+extern void InitializeParallelDSM(ParallelContext *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..5b61ce4 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ bool parallelReacquire);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,9 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(char *start_address);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
Hi,
On 2015-02-06 22:43:21 -0500, Robert Haas wrote:
Here's v4, with that fixed and a few more tweaks.
If you attached files generated with 'git format-patch' I could directly
apply then with the commit message and such. All at once if it's
mutliple patches, as individual commits. On nontrivial patches it's nice
to see the commit message(s) along the diff(s).
Observations:
* Some tailing whitespace in the readme. Very nice otherwise.
* Don't like CreateParallelContextForExtension much as a function name -
I don't think we normally equate the fact that code is located in a
loadable library with the extension mechanism.
* StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ...) what's
that about?
* the plain shm calculations intentionally use mul_size/add_size to deal
with overflows. On 32bit it doesn't seem impossible, but unlikely to
overflow size_t.
* I'd s/very // i "We might be running in a very short-lived memory
context.". Or replace it with "too".
* In +LaunchParallelWorkers, does it make sense trying to start workers
if one failed? ISTM that's likely to not be helpful. I.e. it should
just break; after the first failure.
* +WaitForParallelWorkersToFinish says that it waits for workers to exit
cleanly. To me that's ambiguous. How about "fully"?
* ParallelMain restores libraries before GUC state. Given that
librararies can, and actually somewhat frequently do, inspect GUCs on
load, it seems better to do it the other way round? You argue "We want
to do this before restoring GUCs, because the libraries might define
custom variables.", but I don't buy that. It's completely normal for
namespaced GUCs to be present before a library is loaded? Especially
as you then go and process_session_preload_libraries() after setting
the GUCs.
* Should ParallelMain maybe enter the parallel context before loading
user defined libraries? It's far from absurd to execute code touching
the database on library initialization...
* rename ParallelMain to ParallelWorkerMain?
* I think restoring snapshots needs to fudge the worker's PGXACT->xmin
to be the minimum of the top transaction id and the
snapshots. Otherwise if the initiating process dies badly
(e.g. because postmaster died) the workers will continue to work,
while other processes may remove things. Also, you don't seem to
prohibit popping the active snapshot (should that be prohibitted
maybe?) which might bump the initiator's xmin horizon.
* Comment about 'signal_handler' in +HandleParallelMessageInterrupt
is outdated.
* Is it really a good idea to overlay Z/ReadyForQuery with 'this worker
exited cleanly'? Shouldn't it at least be T/Terminate? I'm generally
not sure it's wise to use this faux FE/BE protocol here...
* HandlingParallelMessages looks dead.
* ParallelErrorContext has the wrong comment.
* ParallelErrorContext() provides the worker's pid in the context
message. I guess that's so it's easier to interpret when sent to the
initiator? It'll look odd when logged by the failing process.
* We now have pretty much the same handle_sigterm in a bunch of
places. Can't we get rid of those? You actually probably can just use
die().
* The comments in xact.c above XactTopTransactionId pretty much assume
that the reader knows that that is about parallel stuff.
* I'm a bit confused by the fact that Start/CommitTransactionCommand()
emit a generic elog when encountering a PARALLEL_INPROGRESS, whereas
ReleaseSavepoint()/RollbackTo has a special message. Shouldn't it be
pretty much impossible to hit a ReleaseSavepoint() with a parallel
transaction in progress? We'd have had to end the previous transaction
command while parallel stuff was in progress - right?
(Internal/ReleaseCurrentSubTransaction is different, that's used in code)
* Why are you deviating from the sorting order used in other places for
ParallelCurrentXids? That seems really wierd, especially as we use
something else a couple lines down. Especially as you actually seem to
send stuff in xidComparator order?
* I don't think skipping AtEOXact_Namespace() entirely if parallel is a
good idea. That function already does some other stuff than cleaning
up temp tables. I think you should pass in parallel and do the skip in
there.
* Start/DndParallelWorkerTransaction assert the current state, whereas
the rest of the file FATALs in that case. I think it'd actually be
good to be conservative and do the same in this case.
* You're manually passing function names to
PreventCommandIfParallelMode() in a fair number of cases. I'd either
try and keep the names consistent with what the functions are actually
called at the sql level (adding the types in the parens) or just use
PG_FUNCNAME_MACRO to keep them correct.
* Wait. You now copy all held relation held "as is" to the standby? I
quite doubt that's a good idea, and it's not my reading of the
conclusions made in the group locking thread. At the very least this
part needs to be extensively documented. And while
LockAcquireExtended() refers to
src/backend/access/transam/README.parallel for details I don't see
anything pertinent in there. And the function header sounds like the
only difference is the HS logging - not mentioning that it essentially
disables lock queuing entirely.
This seems unsafe (e.g. consider if the initiating backend died and
somebody else acquired the lock, possible e.g. if postmaster died) and
not even remotely enough discussed. I think this should be removed
from the patch for now.
Greetings,
Andres Freund
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Sat, Feb 7, 2015 at 7:20 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Observations:
* Some tailing whitespace in the readme. Very nice otherwise.
Fixed. Thanks.
* Don't like CreateParallelContextForExtension much as a function name -
I don't think we normally equate the fact that code is located in a
loadable library with the extension mechanism.
Suggestions for a better name? CreateParallelContextForLoadableFunction?
* StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ...) what's
that about?
Gee, maybe I should have added a comment so I'd remember. If the
buffer size isn't MAXALIGN'd, that would be really bad, because
shm_mq_create() assumes that it's being given an aligned address.
Maybe I should add an Assert() there. If it is MAXALIGN'd but not
BUFFERALIGN'd, we might waste a few bytes of space, since
shm_toc_allocate() always allocates in BUFFERALIGN'd chunks, but I
don't think anything will actually break. Not sure if that's worth an
assert or not.
* the plain shm calculations intentionally use mul_size/add_size to deal
with overflows. On 32bit it doesn't seem impossible, but unlikely to
overflow size_t.
Yes, I do that here too, though as with the plain shm calculations,
only in the estimate functions. The functions that actually serialize
stuff don't have to worry about overflow because it's already been
checked.
* I'd s/very // i "We might be running in a very short-lived memory
context.". Or replace it with "too".
Removed "very".
* In +LaunchParallelWorkers, does it make sense trying to start workers
if one failed? ISTM that's likely to not be helpful. I.e. it should
just break; after the first failure.
It can't just break, because clearing pcxt->worker[i].error_mqh is an
essential step. I could add a flag variable that tracks whether any
registrations have failed and change the if statement to if
(!any_registrations_failed && RegisterDynamicBackgroundWorker(&worker,
&pcxt->worker[i].bgwhandle)), if you want. I thought about doing that
but decided it was very unlikely to affect the real-world performance
of anything, so easier just to keep the code simple. But I'll change
it if you want.
* +WaitForParallelWorkersToFinish says that it waits for workers to exit
cleanly. To me that's ambiguous. How about "fully"?
I've removed the word "cleanly" and added a comment to more fully
explain the danger:
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
* ParallelMain restores libraries before GUC state. Given that
librararies can, and actually somewhat frequently do, inspect GUCs on
load, it seems better to do it the other way round? You argue "We want
to do this before restoring GUCs, because the libraries might define
custom variables.", but I don't buy that. It's completely normal for
namespaced GUCs to be present before a library is loaded? Especially
as you then go and process_session_preload_libraries() after setting
the GUCs.
This is a good question, but the answer is not entirely clear to me.
I'm thinking I should probably just remove
process_session_preload_libraries() altogether at this point. That
was added at a time when RestoreLibraryState() didn't exist yet, and I
think RestoreLibraryState() is a far superior way of handling this,
because surely the list of libraries that the parallel leader
*actually had loaded* is more definitive than any GUC.
Now, that doesn't answer the question about whether we should load
libraries first or GUCs. I agree that the comment's reasoning is
bogus, but I'm not sure I understand why you think the other order is
better. It *is* normal for namespaced GUCs to be present before a
library is loaded, but it's equally normal (and, I think, often more
desirable in practice) to load the library first and then set the
GUCs. Generally, I think that libraries ought to be loaded as early
as possible, because they may install hooks that change the way other
stuff works, and should therefore be loaded before that other stuff
happens.
* Should ParallelMain maybe enter the parallel context before loading
user defined libraries? It's far from absurd to execute code touching
the database on library initialization...
It's hard to judge without specific examples. What kinds of things do
they do? Are they counting on a transaction being active? I would
have thought that was a no-no, since there are many instances in which
it won't be true. Also, you might have just gotten loaded because a
function stored in your library was called, so you could be in a
transaction that's busy doing something else, or deep in a
subtransaction stack, etc. It seems unsafe to do very much more than
a few syscache lookups here, even if there does happen to be a
transaction active.
* rename ParallelMain to ParallelWorkerMain?
Sounds good. Done.
* I think restoring snapshots needs to fudge the worker's PGXACT->xmin
to be the minimum of the top transaction id and the
snapshots. Otherwise if the initiating process dies badly
(e.g. because postmaster died) the workers will continue to work,
while other processes may remove things.
RestoreTransactionSnapshot() works the same way as the existing
import/export snapshot stuff, so that ought to be no less safe than
what we're doing already. Any other snapshots that we're restoring
had better not have an xmin lower than that one; if they do, the
master messed up. Possibly it would be a good idea to have additional
safeguards there; not sure exactly what. Perhaps RestoreSnapshot()
could assert that the xmin of the restored snapshot
follows-or-is-equal-to PGXACT->xmin? That would be safe for the first
snapshot we restore because our xmin will be InvalidTransactionId, and
after that it should check the condition you're worried about?
Thoughts?
Also, you don't seem to
prohibit popping the active snapshot (should that be prohibitted
maybe?) which might bump the initiator's xmin horizon.
I think as long as our transaction snapshot is installed correctly our
xmin horizon can't advance; am I missing something?
It's generally OK to pop the active snapshot, as long as you only pop
what you pushed. In a worker, you can push additional snapshots on
the active snapshot stack and then pop them, but you can't pop the one
ParallelWorkerMain installed for you. You'll probably notice if you
do, because then the snapshot stack will be empty when you get back to
ParallelWorkerMain() and you'll fail an assertion. Similarly, the
master shouldn't pop the snapshot that was active at the start of
parallelism until parallelism is done, but again if you did you'd
probably fail an assertion later on. Generally, we haven't had much
of a problem with PushActiveSnapshot() and PopActiveSnapshot() calls
being unbalanced, so I don't really think this is an area that needs
especially strong cross-checks. At least not unless we get some
evidence that this is a more-common mistake in code that touches
parallelism than it is in general.
* Comment about 'signal_handler' in +HandleParallelMessageInterrupt
is outdated.
Removed.
* Is it really a good idea to overlay Z/ReadyForQuery with 'this worker
exited cleanly'? Shouldn't it at least be T/Terminate? I'm generally
not sure it's wise to use this faux FE/BE protocol here...
Well, I'm not sure about that either and never have been, but I was
even less sure inventing a new one was any better. We might need a
few new protocol messages (or to reuse a few existing ones for other
things) but being able to reuse the existing format for ErrorResponse,
NoticeResponse, etc. seems like a pretty solid win. Those are
reasonably complex message formats and reimplementing them for no
reason seems like a bad idea.
Terminate is 'X', not 'T', and it's a frontend-only message. The
worker is speaking the backend half of the protocol. We could use it
anyway; that wouldn't be silly. I picked ReadyForQuery because that's
what the backend sends when it is completely done processing
everything that the user most recently requested, which seems
defensible here.
* HandlingParallelMessages looks dead.
Good catch. Removed.
* ParallelErrorContext has the wrong comment.
Doh. Fixed.
* ParallelErrorContext() provides the worker's pid in the context
message. I guess that's so it's easier to interpret when sent to the
initiator? It'll look odd when logged by the failing process.
Yes, that's why. Regarding logging, true. I guess the master could
add the context instead, although making sure the PID is available
looks pretty annoying. At the time we establish the queue, the PID
isn't known yet, and by the time we read the error from it, the worker
might be gone, such that we can't read its PID. To fix, we'd have to
invent a new protocol message that means "here's my PID". Another
alternative is to just say that the error came from a parallel worker
(e.g. "while executing in parallel worker") and not mention the PID,
but that seems like it'd be losing possibly-useful information.
* We now have pretty much the same handle_sigterm in a bunch of
places. Can't we get rid of those? You actually probably can just use
die().
Good idea. Done.
* The comments in xact.c above XactTopTransactionId pretty much assume
that the reader knows that that is about parallel stuff.
What would you suggest? The comment begins "Only a single
TransactionStateData is placed on the parallel worker's state stack",
which seems like a pretty clear way of giving the user a hint that we
are talking about parallel stuff. An older version of the patch was
much less clear, but I thought I'd remedied that.
* I'm a bit confused by the fact that Start/CommitTransactionCommand()
emit a generic elog when encountering a PARALLEL_INPROGRESS, whereas
ReleaseSavepoint()/RollbackTo has a special message. Shouldn't it be
pretty much impossible to hit a ReleaseSavepoint() with a parallel
transaction in progress? We'd have had to end the previous transaction
command while parallel stuff was in progress - right?
(Internal/ReleaseCurrentSubTransaction is different, that's used in code)
It's pretty simple to hit ReleaseSavepoint() while a transaction is in
progress. It's pretty much directly SQL-callable, so a PL function
run in a parallel worker could easily hit it, or anything that uses
SPI. There are similar checks in e.g. EndTransaction(), but you can't
invoke CommitTransactionCommand() directly.
* Why are you deviating from the sorting order used in other places for
ParallelCurrentXids? That seems really wierd, especially as we use
something else a couple lines down. Especially as you actually seem to
send stuff in xidComparator order?
The transaction IDs have to be sorted into some order so that they can
be binary-searched, and this seemed simplest. xidComparator sorts in
numerical order, not transaction-ID order, so that's how we send them.
That turns out to be convenient anyway, because binary-search on a
sorted array of integers is really simple. If we sent them sorted in
transaction-ID order we'd have to make sure that the reference
transaction ID was the same for both backends, which might not be
hard, but I don't see how it would be better than this.
* I don't think skipping AtEOXact_Namespace() entirely if parallel is a
good idea. That function already does some other stuff than cleaning
up temp tables. I think you should pass in parallel and do the skip in
there.
That's a very good point. Fixed.
* Start/DndParallelWorkerTransaction assert the current state, whereas
the rest of the file FATALs in that case. I think it'd actually be
good to be conservative and do the same in this case.
Well, actually, StartTransaction() does this:
if (s->state != TRANS_DEFAULT)
elog(WARNING, "StartTransaction while in %s state",
TransStateAsString(s->state));
I could copy and paste that code into StartParallelWorkerTransaction()
and changing WARNING to FATAL, but the first thing
StartParallelWorkerTransaction() does is call StartTransaction(). It
seems pretty stupid to have two identical tests that differ only in
their log level. The same considerations apply to
EndParalellWorkerTransaction() and CommitTransaction().
A worthwhile question is why somebody thought that it was a good idea
for the log level there to be WARNING rather than FATAL. But I don't
think it's this patch's job to second-guess that decision.
* You're manually passing function names to
PreventCommandIfParallelMode() in a fair number of cases. I'd either
try and keep the names consistent with what the functions are actually
called at the sql level (adding the types in the parens) or just use
PG_FUNCNAME_MACRO to keep them correct.
I think putting the type names in is too chatty; I'm not aware we use
that style in other error messages. We don't want to lead people to
believe that only the form with the particular argument types they
used is not OK.
PG_FUNCNAME_MACRO will give us the C name, not the SQL name.
* Wait. You now copy all held relation held "as is" to the standby? I
quite doubt that's a good idea, and it's not my reading of the
conclusions made in the group locking thread. At the very least this
part needs to be extensively documented. And while
LockAcquireExtended() refers to
src/backend/access/transam/README.parallel for details I don't see
anything pertinent in there. And the function header sounds like the
only difference is the HS logging - not mentioning that it essentially
disables lock queuing entirely.This seems unsafe (e.g. consider if the initiating backend died and
somebody else acquired the lock, possible e.g. if postmaster died) and
not even remotely enough discussed. I think this should be removed
from the patch for now.
If it's broken, we need to identify what's wrong and fix it, not just
rip it out. It's possible that something is broken with that code,
but it's dead certain that something is broken without it:
rhaas=# select parallel_count('pgbench_accounts', 1);
NOTICE: PID 57956 counted 2434815 tuples
NOTICE: PID 57957 counted 1565185 tuples
CONTEXT: parallel worker, pid 57957
parallel_count
----------------
4000000
(1 row)
rhaas=# begin;
BEGIN
rhaas=# lock pgbench_accounts;
LOCK TABLE
rhaas=# select parallel_count('pgbench_accounts', 1);
NOTICE: PID 57956 counted 4000000 tuples
...and then it hangs forever.
On the specific issues:
1. I agree that it's very dangerous for the parallel backend to
acquire the lock this way if the master no longer holds it.
Originally, I was imagining that there would be no interlock between
the master shutting down and the worker starting up, but you and
others convinced me that was a bad idea. So now transaction commit or
abort waits for all workers to be gone, which I think reduces the
scope of possible problems here pretty significantly. However, it's
quite possible that it isn't airtight. One thing we could maybe do to
make it safer is pass a pointer to the initiator's PGPROC. If we get
the lock via the fast-path we are safe anyway, but if we have to
acquire the partition lock, then we can cross-check that the
initiator's lock is still there. I think that would button this up
pretty tight.
2. My reading of the group locking discussions was that everybody
agreed that the originally-proposed group locking approach, which
involved considering locks from the same locking group as mutually
non-conflicting, was not OK. Several specific examples were offered -
e.g. it's clearly not OK for two backends to extend a relation at the
same time just because the same locking group. So I abandoned that
approach. When I instead proposed the approach of copying only the
locks that the master *already* had at the beginning of parallelism
and considering *only those* as mutually conflicting, I believe I got
several comments to the effect that this was "less scary".
Considering the topic area, I'm not sure I'm going to do any better
than that.
3. I welcome proposals for other ways of handling this problem, even
if they restrict the functionality that can be offered. For example,
a proposal that would make parallel_count revert to single-threaded
mode but terminate without an indefinite wait would be acceptable to
me, provided that it offers some advantage in safety and security over
what I've already got. A proposal to make it parallel_count error out
in the above case would not be acceptable to me; the planner must not
generate parallel plans that will sometimes fail unexpectedly at
execution-time. I generally believe that we will be much happier if
application programmers need not worry about the failure of parallel
workers to obtain locks already held by the master; some such failure
modes may be very complex and hard to predict. The fact that the
current approach handles the problem entirely within the lock manager,
combined with the fact that it is extremely simple, is therefore very
appealing to me. Nonetheless, a test case that demonstrates this
approach falling down badly would force a rethink; do you have one?
Or an idea about what it might look 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 2015-02-10 11:49:58 -0500, Robert Haas wrote:
On Sat, Feb 7, 2015 at 7:20 PM, Andres Freund <andres@2ndquadrant.com> wrote:
* Don't like CreateParallelContextForExtension much as a function name -
I don't think we normally equate the fact that code is located in a
loadable library with the extension mechanism.Suggestions for a better name? CreateParallelContextForLoadableFunction?
*ExternalFunction maybe? That's dfmgr.c calls it.
* the plain shm calculations intentionally use mul_size/add_size to deal
with overflows. On 32bit it doesn't seem impossible, but unlikely to
overflow size_t.Yes, I do that here too, though as with the plain shm calculations,
only in the estimate functions. The functions that actually serialize
stuff don't have to worry about overflow because it's already been
checked.
I thought I'd seen some estimation functions that didn't use it. But
apparently I was wrong.
* In +LaunchParallelWorkers, does it make sense trying to start workers
if one failed? ISTM that's likely to not be helpful. I.e. it should
just break; after the first failure.It can't just break, because clearing pcxt->worker[i].error_mqh is an
essential step. I could add a flag variable that tracks whether any
registrations have failed and change the if statement to if
(!any_registrations_failed && RegisterDynamicBackgroundWorker(&worker,
&pcxt->worker[i].bgwhandle)), if you want. I thought about doing that
but decided it was very unlikely to affect the real-world performance
of anything, so easier just to keep the code simple. But I'll change
it if you want.
I think it'd be better.
* ParallelMain restores libraries before GUC state. Given that
librararies can, and actually somewhat frequently do, inspect GUCs on
load, it seems better to do it the other way round? You argue "We want
to do this before restoring GUCs, because the libraries might define
custom variables.", but I don't buy that. It's completely normal for
namespaced GUCs to be present before a library is loaded? Especially
as you then go and process_session_preload_libraries() after setting
the GUCs.This is a good question, but the answer is not entirely clear to me.
I'm thinking I should probably just remove
process_session_preload_libraries() altogether at this point. That
was added at a time when RestoreLibraryState() didn't exist yet, and I
think RestoreLibraryState() is a far superior way of handling this,
because surely the list of libraries that the parallel leader
*actually had loaded* is more definitive than any GUC.
That sounds like a good idea to me.
Now, that doesn't answer the question about whether we should load
libraries first or GUCs. I agree that the comment's reasoning is
bogus, but I'm not sure I understand why you think the other order is
better. It *is* normal for namespaced GUCs to be present before a
library is loaded, but it's equally normal (and, I think, often more
desirable in practice) to load the library first and then set the
GUCs.
Well, it's pretty much never the case that the library is loaded before
postgresql.conf gucs, right? A changed postgresql.conf is the only
exception I can see. Neither is it the normal case for
session|local_preload_libraries. Not even when GUCs are loaded via
pg_db_role_setting or the startup packet...
Generally, I think that libraries ought to be loaded as early
as possible, because they may install hooks that change the way other
stuff works, and should therefore be loaded before that other stuff
happens.
While that may be desirable I don't really see a reason for this to be
treated differently for worker processes than the majority of cases
otherwise.
Anyway, I think this is a relatively minor issue.
* Should ParallelMain maybe enter the parallel context before loading
user defined libraries? It's far from absurd to execute code touching
the database on library initialization...It's hard to judge without specific examples. What kinds of things do
they do?
I've seen code filling lookup caches and creating system objects
(including tables and extensions).
Are they counting on a transaction being active?
I would have thought that was a no-no, since there are many instances
in which it won't be true. Also, you might have just gotten loaded
because a function stored in your library was called, so you could be
in a transaction that's busy doing something else, or deep in a
subtransaction stack, etc. It seems unsafe to do very much more than
a few syscache lookups here, even if there does happen to be a
transaction active.
The only reason I'd like it to be active is because that'd *prohibit*
doing the crazier stuff. There seems little reason to not da it under
the additional protection against crazy things that'd give us?
* I think restoring snapshots needs to fudge the worker's PGXACT->xmin
to be the minimum of the top transaction id and the
snapshots. Otherwise if the initiating process dies badly
(e.g. because postmaster died) the workers will continue to work,
while other processes may remove things.RestoreTransactionSnapshot() works the same way as the existing
import/export snapshot stuff, so that ought to be no less safe than
what we're doing already.
Well, ExportSnapshot()/Import has quite a bit more restrictions than
what you're doing... Most importantly it cannot import in transactions
unless using read committed isolation, forces xid assignment during
export, forces the old snapshot to stick around for the whole
transaction and only works on a primary. I'm not actually sure it makes
a relevant difference, but it's certainly worth calling attention to.
The fuding I was wondering about certainly is unnecessary though -
GetSnapshotData() has already performed it...
I'm not particularly awake right now and I think this needs a closer
look either by someone awake... I'm not fully convinced this is safe.
Also, you don't seem to
prohibit popping the active snapshot (should that be prohibitted
maybe?) which might bump the initiator's xmin horizon.I think as long as our transaction snapshot is installed correctly our
xmin horizon can't advance; am I missing something?
Maybe I'm missing something, but why would that be the case in a read
committed session? ImportSnapshot() only ever calls
SetTransactionSnapshot it such a case (why does it contain code to cope
without it?), but your patch doesn't seem to guarantee that.
But as don't actually transport XactIsoLevel anywhere (omission
probably?) that seems to mean that the SetTransactionSnapshot() won't do
much, even if the source transaction is repeatable read.
Now the thing that actually does give some guarantees is that you
immediately afterwards restore the active snapshot and do a
PushActiveSnapshot(), which'll prevent MyPgXact->xmin from changing
because SnapshotResetXmin() refuses to do anything if there's an
ActiveSnapshot. Seems a bit comlicated and fragile though.
* Is it really a good idea to overlay Z/ReadyForQuery with 'this worker
exited cleanly'? Shouldn't it at least be T/Terminate? I'm generally
not sure it's wise to use this faux FE/BE protocol here...Well, I'm not sure about that either and never have been, but I was
even less sure inventing a new one was any better. We might need a
few new protocol messages (or to reuse a few existing ones for other
things) but being able to reuse the existing format for ErrorResponse,
NoticeResponse, etc. seems like a pretty solid win. Those are
reasonably complex message formats and reimplementing them for no
reason seems like a bad idea.Terminate is 'X', not 'T'
Oops, yes.
and it's a frontend-only message. The worker is speaking the backend
half of the protocol. We could use it anyway; that wouldn't be silly.
Even if it's a frontend only one it doesn't seem like a bad idea. I've
occasionally wished the backend would send explicit termination messages
in a bunch of scenarios. I'm a bit tired of seing:
FATAL: 57P01: terminating connection due to administrator command
LOCATION: ProcessInterrupts, postgres.c:2888
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: Succeeded.
wheen the session was terminated while a query is active. A explicit
termination message seems like a nicer solution than interpreting the
FATAL severity.
I picked ReadyForQuery because that's
what the backend sends when it is completely done processing
everything that the user most recently requested, which seems
defensible here.
I'm pretty sure that we're going to reuse workers within a parallel
query at some point and ready for query seems like a nicer code for
saying 'finished with my work, give me the next thing'.
* ParallelErrorContext() provides the worker's pid in the context
message. I guess that's so it's easier to interpret when sent to the
initiator? It'll look odd when logged by the failing process.Yes, that's why. Regarding logging, true. I guess the master could
add the context instead, although making sure the PID is available
looks pretty annoying. At the time we establish the queue, the PID
isn't known yet, and by the time we read the error from it, the worker
might be gone, such that we can't read its PID. To fix, we'd have to
invent a new protocol message that means "here's my PID".
Hm. Or we could attach the pid to the error message in that case - just
like there already is schema_name etc. Or, to stay more in the FE/BE
vibe - we could just send a 'K' message at startup which sends
MyProcPid, MyCancelKey in normal connections.
* The comments in xact.c above XactTopTransactionId pretty much assume
that the reader knows that that is about parallel stuff.What would you suggest? The comment begins "Only a single
TransactionStateData is placed on the parallel worker's state stack",
which seems like a pretty clear way of giving the user a hint that we
are talking about parallel stuff.
Replacing "Only" by "When running as parallel worker we only place a
..." would already help. To me the comment currently readss like it
desperately wishes to be located in the function initiating parallelism
rather than global file scope. Maybe it's lonely or such.
* I'm a bit confused by the fact that Start/CommitTransactionCommand()
emit a generic elog when encountering a PARALLEL_INPROGRESS, whereas
ReleaseSavepoint()/RollbackTo has a special message. Shouldn't it be
pretty much impossible to hit a ReleaseSavepoint() with a parallel
transaction in progress? We'd have had to end the previous transaction
command while parallel stuff was in progress - right?
(Internal/ReleaseCurrentSubTransaction is different, that's used in code)It's pretty simple to hit ReleaseSavepoint() while a transaction is in
progress. It's pretty much directly SQL-callable, so a PL function
run in a parallel worker could easily hit it, or anything that uses
SPI.
SPI doesn't really work across subtransaction boundaries, so it really
should never be called that way. Which is way it (and thus the PLs)
return SPI_ERROR_TRANSACTION in case you try to execute it. If it's
possible to hit it, we have a problem - I think it'd pretty much lead to
rampant memory clobbering and such. Now, I'm not against providing a
error check, I was just surprised about the verbosity of the different
locations.
* Why are you deviating from the sorting order used in other places for
ParallelCurrentXids? That seems really wierd, especially as we use
something else a couple lines down. Especially as you actually seem to
send stuff in xidComparator order?The transaction IDs have to be sorted into some order so that they can
be binary-searched, and this seemed simplest. xidComparator sorts in
numerical order, not transaction-ID order, so that's how we send them.
Forget that bit. Was tired. I'm not sure what I thought.
* Start/DndParallelWorkerTransaction assert the current state, whereas
the rest of the file FATALs in that case. I think it'd actually be
good to be conservative and do the same in this case.Well, actually, StartTransaction() does this:
if (s->state != TRANS_DEFAULT)
elog(WARNING, "StartTransaction while in %s state",
TransStateAsString(s->state));
But StartTransactionCommand does
elog(ERROR, "StartTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
and CommitTransactionCommand does
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
and some more. These only deal with blockState though...
A worthwhile question is why somebody thought that it was a good idea
for the log level there to be WARNING rather than FATAL.
Yea, it's really rather odd. Everytime I've seen those WARNINGS, e.g. in
the recent "hung backends stuck in spinlock" things got even more
pearshaped shortly afterwards.
But I don't think it's this patch's job to second-guess that decision.
Fair enough.
* You're manually passing function names to
PreventCommandIfParallelMode() in a fair number of cases. I'd either
try and keep the names consistent with what the functions are actually
called at the sql level (adding the types in the parens) or just use
PG_FUNCNAME_MACRO to keep them correct.I think putting the type names in is too chatty; I'm not aware we use
that style in other error messages. We don't want to lead people to
believe that only the form with the particular argument types they
used is not OK.
PG_FUNCNAME_MACRO will give us the C name, not the SQL name.
Your manual ones don't either, that's what made me
complain. E.g. pg_advisory_lock_int8 isn't called that on the SQL
level. Instead they use the C name + parens.
* Wait. You now copy all held relation held "as is" to the standby? I
quite doubt that's a good idea, and it's not my reading of the
conclusions made in the group locking thread. At the very least this
part needs to be extensively documented. And while
LockAcquireExtended() refers to
src/backend/access/transam/README.parallel for details I don't see
anything pertinent in there. And the function header sounds like the
only difference is the HS logging - not mentioning that it essentially
disables lock queuing entirely.This seems unsafe (e.g. consider if the initiating backend died and
somebody else acquired the lock, possible e.g. if postmaster died) and
not even remotely enough discussed. I think this should be removed
from the patch for now.If it's broken, we need to identify what's wrong and fix it, not just
rip it out. It's possible that something is broken with that code,
but it's dead certain that something is broken without it:rhaas=# select parallel_count('pgbench_accounts', 1);
NOTICE: PID 57956 counted 2434815 tuples
NOTICE: PID 57957 counted 1565185 tuples
CONTEXT: parallel worker, pid 57957
parallel_count
----------------
4000000
(1 row)rhaas=# begin;
BEGIN
rhaas=# lock pgbench_accounts;
LOCK TABLE
rhaas=# select parallel_count('pgbench_accounts', 1);
NOTICE: PID 57956 counted 4000000 tuples...and then it hangs forever.
Which is *not* a good example for the problem. Your primary reasoning
for needing something more than sharing the locks that we know the
individual query is going to need (which we acquire in the parse
analzye/planner/executor) is that we can't predict what some random code
does. Now, I still don't think that argument should hold too much sway
because we'll only be able to run carefully controlled code
*anyway*. But *if* we take it as reasoning for doing more than granting
the locks we need for the individual query, we *still* need to cope with
exactly the variants of the above invisible deadlock. You can still can
call a function acquiring some form of lock on either side.
If you'd, as I've argued for before, provide a API that granted workers
precisely the locks needed for the execution of a certain type of
parallel action, I'd be happy. I.e. regular parallel queries would
transport an extended version of what InitPlan() does with relation
lock.
On the specific issues:
1. I agree that it's very dangerous for the parallel backend to
acquire the lock this way if the master no longer holds it.
Originally, I was imagining that there would be no interlock between
the master shutting down and the worker starting up, but you and
others convinced me that was a bad idea. So now transaction commit or
abort waits for all workers to be gone, which I think reduces the
scope of possible problems here pretty significantly. However, it's
quite possible that it isn't airtight. One thing we could maybe do to
make it safer is pass a pointer to the initiator's PGPROC. If we get
the lock via the fast-path we are safe anyway, but if we have to
acquire the partition lock, then we can cross-check that the
initiator's lock is still there. I think that would button this up
pretty tight.
That'd certainly make me feel less bad about it.
2. My reading of the group locking discussions was that everybody
agreed that the originally-proposed group locking approach, which
involved considering locks from the same locking group as mutually
non-conflicting, was not OK. Several specific examples were offered -
e.g. it's clearly not OK for two backends to extend a relation at the
same time just because the same locking group. So I abandoned that
approach. When I instead proposed the approach of copying only the
locks that the master *already* had at the beginning of parallelism
and considering *only those* as mutually conflicting, I believe I got
several comments to the effect that this was "less scary".
Considering the topic area, I'm not sure I'm going to do any better
than that.
It surely is less scary, but still darn scary. It's just friggin hard to
have a mental model where (self) exclusive locks suddenly aren't that
anymore. It'll get more and more dangerous the more we start to relax
restrictions around parallelism - and that *will* come.
This concern does not only apply to user level commands, but also our
own C code. We will find more and more reasons to parallelize commands
and if we will have problems with suddenly not having a chance to
prevent parallelism during certain actions. Say we parallelize VACUUM
(grand idea for the index scans!) and someone wants to also improve the
truncation. Suddenly the AEL during truncation doesn't give us
guarantees anymore. There'll be many more of these, and we can't really
reason about them because we used to working locks.
3. I welcome proposals for other ways of handling this problem, even
if they restrict the functionality that can be offered. For example,
a proposal that would make parallel_count revert to single-threaded
mode but terminate without an indefinite wait would be acceptable to
me, provided that it offers some advantage in safety and security over
what I've already got.
I think the above example where we only grant the locks required for a
specific thing and only on the relevant severity would already be a big
improvement. Even better would be to to use the computed set of locks to
check whether workers could acquire them and refuse paralellism in that
case.
Another thing to do is to mark the lock, both in the master and workers,
as not effectively being of the original severity anymore. If the own
process again tries to acquire the lock on a heavier severity than what
was needed at the time of query execution, error out. At least until
parallel mode has confirmed to have ended. That should pretty much never
happen.
I don't see how we can avoid teaching the deadlock detector about all
these silent deadlocks in any cases. By your own reasoning.
A proposal to make it parallel_count error out
in the above case would not be acceptable to me; the planner must not
generate parallel plans that will sometimes fail unexpectedly at
execution-time. I generally believe that we will be much happier if
application programmers need not worry about the failure of parallel
workers to obtain locks already held by the master; some such failure
modes may be very complex and hard to predict. The fact that the
current approach handles the problem entirely within the lock manager,
combined with the fact that it is extremely simple, is therefore very
appealing to me.
It's trivial to be simple and unsafe...
Nonetheless, a test case that demonstrates this approach falling down
badly would force a rethink; do you have one? Or an idea about what
it might look like?
No, I don't have one handy. I have the very strong gut feeling that this
would introduce a severe architectural debt that we won't be able to get
rid of afterwards, even if it proves to be a really bad idea. To me
it's taking a shortcut directly through the hard problems.
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 8:45 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Suggestions for a better name? CreateParallelContextForLoadableFunction?
*ExternalFunction maybe? That's dfmgr.c calls it.
Done.
It can't just break, because clearing pcxt->worker[i].error_mqh is an
essential step. I could add a flag variable that tracks whether any
registrations have failed and change the if statement to if
(!any_registrations_failed && RegisterDynamicBackgroundWorker(&worker,
&pcxt->worker[i].bgwhandle)), if you want. I thought about doing that
but decided it was very unlikely to affect the real-world performance
of anything, so easier just to keep the code simple. But I'll change
it if you want.I think it'd be better.
Done.
This is a good question, but the answer is not entirely clear to me.
I'm thinking I should probably just remove
process_session_preload_libraries() altogether at this point. That
was added at a time when RestoreLibraryState() didn't exist yet, and I
think RestoreLibraryState() is a far superior way of handling this,
because surely the list of libraries that the parallel leader
*actually had loaded* is more definitive than any GUC.That sounds like a good idea to me.
Done.
Well, it's pretty much never the case that the library is loaded before
postgresql.conf gucs, right? A changed postgresql.conf is the only
exception I can see. Neither is it the normal case for
session|local_preload_libraries. Not even when GUCs are loaded via
pg_db_role_setting or the startup packet...
Well, it can happen when you issue an explicit LOAD, or an implicit
one by calling a function that's in that module, and then go back and
set the GUC afterward, if nothing else.
Anyway, I think this is a relatively minor issue.
I'm going to leave this alone for now. Evidence may emerge that this
is better done in the other order, but in the absence of evidence I'm
going to follow my gut rather than yours. With all due respect for
your gut, of course.
The only reason I'd like it to be active is because that'd *prohibit*
doing the crazier stuff. There seems little reason to not da it under
the additional protection against crazy things that'd give us?
Trying to load additional libraries once parallel mode is in progress
can provide failures, because the load of the libraries causes the
system to do GUC_ACTION_SET on the GUCs whose initialization was
deferred, and that trips up the
no-changing-things-while-in-parallel-mode checks. I'm not sure if
there's anything we can, or should, do about that.
Well, ExportSnapshot()/Import has quite a bit more restrictions than
what you're doing... Most importantly it cannot import in transactions
unless using read committed isolation, forces xid assignment during
export, forces the old snapshot to stick around for the whole
transaction and only works on a primary. I'm not actually sure it makes
a relevant difference, but it's certainly worth calling attention to.The fuding I was wondering about certainly is unnecessary though -
GetSnapshotData() has already performed it...I'm not particularly awake right now and I think this needs a closer
look either by someone awake... I'm not fully convinced this is safe.
I'm not 100% comfortable with it either, but I just spent some time
looking at it and can't see what's wrong with it. Basically, we're
trying to get the parallel worker into a state that matches the
master's state after doing GetTransactionSnapshot() - namely,
CurrentSnapshot should point to the same snapshot on the master, and
FirstSnapshotSet should be true, plus the same additional processing
that GetTransactionSnapshot() would have done if we're in a higher
transaction isolation level. It's possible we don't need to mimic
that state, but it seems like a good idea.
Still, I wonder if we ought to be banning GetTransactionSnapshot()
altogether. I'm not sure if there's ever a time when it's safe for a
worker to call that.
Also, you don't seem to
prohibit popping the active snapshot (should that be prohibitted
maybe?) which might bump the initiator's xmin horizon.I think as long as our transaction snapshot is installed correctly our
xmin horizon can't advance; am I missing something?Maybe I'm missing something, but why would that be the case in a read
committed session? ImportSnapshot() only ever calls
SetTransactionSnapshot it such a case (why does it contain code to cope
without it?), but your patch doesn't seem to guarantee that.But as don't actually transport XactIsoLevel anywhere (omission
probably?) that seems to mean that the SetTransactionSnapshot() won't do
much, even if the source transaction is repeatable read.
Doesn't XactIsoLevel get set by assign_XactIsoLevel() when we restore
the GUC state?
Now the thing that actually does give some guarantees is that you
immediately afterwards restore the active snapshot and do a
PushActiveSnapshot(), which'll prevent MyPgXact->xmin from changing
because SnapshotResetXmin() refuses to do anything if there's an
ActiveSnapshot. Seems a bit comlicated and fragile though.
Suggestions?
Terminate is 'X', not 'T'
Oops, yes.
and it's a frontend-only message. The worker is speaking the backend
half of the protocol. We could use it anyway; that wouldn't be silly.Even if it's a frontend only one it doesn't seem like a bad idea.
Done.
I picked ReadyForQuery because that's
what the backend sends when it is completely done processing
everything that the user most recently requested, which seems
defensible here.I'm pretty sure that we're going to reuse workers within a parallel
query at some point and ready for query seems like a nicer code for
saying 'finished with my work, give me the next thing'.
Yeah, could well be.
Hm. Or we could attach the pid to the error message in that case - just
like there already is schema_name etc. Or, to stay more in the FE/BE
vibe - we could just send a 'K' message at startup which sends
MyProcPid, MyCancelKey in normal connections.
Done.
Replacing "Only" by "When running as parallel worker we only place a
..." would already help. To me the comment currently readss like it
desperately wishes to be located in the function initiating parallelism
rather than global file scope. Maybe it's lonely or such.
Heh, heh. Done.
* You're manually passing function names to
PreventCommandIfParallelMode() in a fair number of cases. I'd either
try and keep the names consistent with what the functions are actually
called at the sql level (adding the types in the parens) or just use
PG_FUNCNAME_MACRO to keep them correct.I think putting the type names in is too chatty; I'm not aware we use
that style in other error messages. We don't want to lead people to
believe that only the form with the particular argument types they
used is not OK.PG_FUNCNAME_MACRO will give us the C name, not the SQL name.
Your manual ones don't either, that's what made me
complain. E.g. pg_advisory_lock_int8 isn't called that on the SQL
level. Instead they use the C name + parens.
On further reflection, I think I should probably just change all of
those to use a general message about advisory locks, i.e.:
ERROR: cannot use advisory locks during a parallel operation
That sound good to you?
Which is *not* a good example for the problem. Your primary reasoning
for needing something more than sharing the locks that we know the
individual query is going to need (which we acquire in the parse
analzye/planner/executor) is that we can't predict what some random code
does. Now, I still don't think that argument should hold too much sway
because we'll only be able to run carefully controlled code
*anyway*.
I'll avoid repeating what I've said about this before, except to say
that I still don't believe for a minute you can predict which locks
you'll need.
But *if* we take it as reasoning for doing more than granting
the locks we need for the individual query, we *still* need to cope with
exactly the variants of the above invisible deadlock. You can still can
call a function acquiring some form of lock on either side.
That by itself is not a deadlock. If the worker blocks trying to
acquire a lock that the master held before the start of parallelism,
it will wait forever, even if the master never acquires a lock. But
if the worker blocks trying to acquire a lock that the master acquired
*during* parallelism, it's presumably something like a catalog lock or
a tuple lock or a relation extension lock that the master is going to
release quickly. So the master will finish with that resource and
then the worker will go ahead. You're right that there are other
scenarios to consider, but you need more than that.
On the specific issues:
1. I agree that it's very dangerous for the parallel backend to
acquire the lock this way if the master no longer holds it.
Originally, I was imagining that there would be no interlock between
the master shutting down and the worker starting up, but you and
others convinced me that was a bad idea. So now transaction commit or
abort waits for all workers to be gone, which I think reduces the
scope of possible problems here pretty significantly. However, it's
quite possible that it isn't airtight. One thing we could maybe do to
make it safer is pass a pointer to the initiator's PGPROC. If we get
the lock via the fast-path we are safe anyway, but if we have to
acquire the partition lock, then we can cross-check that the
initiator's lock is still there. I think that would button this up
pretty tight.That'd certainly make me feel less bad about it.
Done.
2. My reading of the group locking discussions was that everybody
agreed that the originally-proposed group locking approach, which
involved considering locks from the same locking group as mutually
non-conflicting, was not OK. Several specific examples were offered -
e.g. it's clearly not OK for two backends to extend a relation at the
same time just because the same locking group. So I abandoned that
approach. When I instead proposed the approach of copying only the
locks that the master *already* had at the beginning of parallelism
and considering *only those* as mutually conflicting, I believe I got
several comments to the effect that this was "less scary".
Considering the topic area, I'm not sure I'm going to do any better
than that.It surely is less scary, but still darn scary. It's just friggin hard to
have a mental model where (self) exclusive locks suddenly aren't that
anymore. It'll get more and more dangerous the more we start to relax
restrictions around parallelism - and that *will* come.
Assuming we get a first version committed at some point, yes.
This concern does not only apply to user level commands, but also our
own C code. We will find more and more reasons to parallelize commands
and if we will have problems with suddenly not having a chance to
prevent parallelism during certain actions. Say we parallelize VACUUM
(grand idea for the index scans!) and someone wants to also improve the
truncation. Suddenly the AEL during truncation doesn't give us
guarantees anymore
If the VACUUM leader process tries to grab an AccessExclusiveLock
after starting parallel mode and before terminating it, surely that's
going to conflict with the locks held by any remaining workers at that
point. Even if it weren't going to conflict, you'd have to be pretty
stupid to code it that way, because you can't remove dead line
pointers until you've finished all of the index scans, and you can't
very well truncate the relation until you've removed dead line
pointers. So to me, this seems like an impossible scenario that
wouldn't break even if it were possible.
There'll be many more of these, and we can't really
reason about them because we used to working locks.
FUD.
I think one thing that might help allay your concerns in this area to
a degree is to discuss under what circumstances we think it's safe to
create a parallel context in the first place. For example, if you
were to insert code that tries to go parallel in the middle of
heap_update() or in the middle of relation extension, that's probably
not going to work out well with this model, or with any model. It's
not *impossible* for it to work out well - if you wanted to extend the
relation by a whole bunch of blocks, the worker could do that while
the master goes and try to read from CLOG or something, but you'd have
to be awfully careful and it's probably a stupid idea for lots of
reasons. It really only makes sense to enter parallelism when the
backend is in a relatively quiescent state - like at the beginning of
a query, or after vacuum scans the heap but before it scans the
indexes. At that point it's a lot easier to reason about what the
parallel backends can safely do.
Broadly, I think that's there's a close connection between what state
the master is in when we initiate parallelism and what the workers can
safely do. I'm not sure exactly how to describe what the connection
is there, but I think it exists, and firming up the way we think about
that might make it easier to reason about this.
3. I welcome proposals for other ways of handling this problem, even
if they restrict the functionality that can be offered. For example,
a proposal that would make parallel_count revert to single-threaded
mode but terminate without an indefinite wait would be acceptable to
me, provided that it offers some advantage in safety and security over
what I've already got.I think the above example where we only grant the locks required for a
specific thing and only on the relevant severity would already be a big
improvement. Even better would be to to use the computed set of locks to
check whether workers could acquire them and refuse paralellism in that
case.
I think that will just produce lots of very-hard-to-debug undetected
deadlocks and huge volumes of code that tries and fails to assess what
locks the worker will need.
Another thing to do is to mark the lock, both in the master and workers,
as not effectively being of the original severity anymore. If the own
process again tries to acquire the lock on a heavier severity than what
was needed at the time of query execution, error out. At least until
parallel mode has confirmed to have ended. That should pretty much never
happen.
I'm not sure what you mean by the "severity" of the lock. How about
marking the locks that the worker inherited from the parallel master
and throwing an error if it tries to lock one of those objects in a
mode that it does not already hold? That'd probably catch a sizeable
number of programming errors without tripping up many legitimate use
cases (and we can always relax or modify the prohibition if it later
turns out to be problematic).
I don't see how we can avoid teaching the deadlock detector about all
these silent deadlocks in any cases. By your own reasoning.
We're not seeing eye to eye here yet, since I don't accept your
example scenario and you don't accept mine. Let's keep discussing.
Meanwhile, here's an updated patch.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v5.patchapplication/x-patch; name=parallel-mode-v5.patchDownload
From 1d501763b1d7f1abc7facee9cfa89521606bc4c0 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Simon
Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 42 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 208 +++++++
src/backend/access/transam/parallel.c | 920 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 451 +++++++++++++-
src/backend/catalog/namespace.c | 4 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 117 +++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 20 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 +++
src/backend/utils/time/snapmgr.c | 210 ++++++-
src/include/access/parallel.h | 63 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 8 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
39 files changed, 2445 insertions(+), 52 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 46060bc1..50bede8 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..b1f8596
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,208 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - End-of-transaction namespace processing is not done. If a pg_temp
+ namespace needs to be cleaned up, the master is responsible for this.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..5bf546c
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,920 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..06b6f7e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,48 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +924,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1835,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1966,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2002,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2012,10 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(true);
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2055,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2088,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2137,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2162,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2188,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2259,7 +2409,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2284,6 +2434,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2455,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2504,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2374,6 +2529,13 @@ AbortTransaction(void)
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
/*
+ * If we might have parallel workers, send them all termination signals,
+ * and wait for them to die.
+ */
+ if (IsInParallelMode())
+ AtEOXact_Parallel(false);
+
+ /*
* do abort processing
*/
AfterTriggerEndXact(false); /* 'false' means it's abort */
@@ -2385,9 +2547,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2572,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2596,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2648,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2705,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2741,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3029,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3419,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3597,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3700,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3739,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3771,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3806,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3841,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3919,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3955,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4044,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4080,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4113,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4147,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4166,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4242,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4334,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4438,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4598,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4745,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4502,6 +4793,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4990,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index bfb4fdc..76f43d8 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -3708,7 +3708,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3718,7 +3718,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 33b172b..07526e8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..0a04dbf 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -669,7 +669,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +681,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -797,7 +805,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
@@ -910,7 +918,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +927,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -3557,6 +3586,86 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 28af40c..8899448 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2989,7 +2990,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..21d9f73d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -420,6 +420,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +438,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +455,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +473,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +493,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +514,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +534,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +555,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +575,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +595,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int8()");
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +613,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +632,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +650,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +669,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventCommandIfParallelMode("pg_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +690,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +712,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +733,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +755,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventCommandIfParallelMode("pg_try_advisory_xact_lock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +776,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +797,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventCommandIfParallelMode("pg_advisory_unlock_shared_int4()");
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 1b69322..affb23a 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -697,3 +698,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9572777..de988ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5602,6 +5602,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6906,6 +6920,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..0685e64
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,63 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index d2e5198..3112cd9 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..b931568 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,9 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
On Wed, Feb 11, 2015 at 1:59 PM, Robert Haas <robertmhaas@gmail.com> wrote:
I'm not sure what you mean by the "severity" of the lock. How about
marking the locks that the worker inherited from the parallel master
and throwing an error if it tries to lock one of those objects in a
mode that it does not already hold? That'd probably catch a sizeable
number of programming errors without tripping up many legitimate use
cases (and we can always relax or modify the prohibition if it later
turns out to be problematic).
Or maybe do that only when the new lock mode is stronger than one it
already holds.
--
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 3:59 AM, Robert Haas <robertmhaas@gmail.com> wrote:
We're not seeing eye to eye here yet, since I don't accept your
example scenario and you don't accept mine. Let's keep discussing.Meanwhile, here's an updated patch.
A lot of cool activity is showing up here, so moved the patch to CF
2015-02. Perhaps Andres you could add yourself as a reviewer?
--
Michael
On Fri, Feb 13, 2015 at 2:22 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:
On Thu, Feb 12, 2015 at 3:59 AM, Robert Haas <robertmhaas@gmail.com> wrote:
We're not seeing eye to eye here yet, since I don't accept your
example scenario and you don't accept mine. Let's keep discussing.Meanwhile, here's an updated patch.
A lot of cool activity is showing up here, so moved the patch to CF 2015-02.
Perhaps Andres you could add yourself as a reviewer?
Here's a new version of the patch with (a) some additional
restrictions on heavyweight locking both at the start of, and during,
parallel mode and (b) a write-up in the README explaining the
restrictions and my theory of why the handling of heavyweight locking
is safe. Hopefully this goes some way towards addressing Andres's
concerns. I've also replaced the specific (and wrong) messages about
advisory locks with a more generic message, as previously proposed;
and I've fixed at least one bug.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v6.patchapplication/x-patch; name=parallel-mode-v6.patchDownload
From f4e815446584bd1930069dda5512f08002e7c07f Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/4] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Simon
Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 42 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 276 +++++++++
src/backend/access/transam/parallel.c | 922 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 459 +++++++++++++-
src/backend/catalog/namespace.c | 4 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 ++++++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 +++
src/backend/utils/time/snapmgr.c | 210 ++++++-
src/include/access/parallel.h | 63 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 10 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
39 files changed, 2651 insertions(+), 52 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 46060bc1..50bede8 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..90f144e
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,276 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated paralellism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+ - Locks on relations and database objects, and any lock held on the transaction's
+ XID. See lock management, below.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initating
+backend to exit any transaction or subtransaction that was in progress when
+paralellism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is
+ responsible for this, too.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+master holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and perhaps the lock
+on our transaction ID are copied to workers. Advisory locks are not copied,
+but the master may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe to
+attempt parallelism while holding a lock of any other type, such as a page,
+tuple, or relation extension lock, and such attempts will fail. Although
+there is currently no reason to do so, such locks could be taken and released
+during parallel mode; they merely cannot be held at the start of parallel
+mode, since we would then fail to provide necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the paralell processe and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..8d39bf2
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,922 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ CheckForRetainedParallelLocks();
+
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 97000ef..e5cd81f 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,51 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+
+ if (s->parallelModeLevel == 0)
+ CheckForRetainedParallelLocks();
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +927,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1705,6 +1838,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1834,6 +1969,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1867,7 +2005,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1876,6 +2015,14 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ CheckForRetainedParallelLocks();
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1915,9 +2062,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1944,7 +2095,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1992,7 +2144,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2017,6 +2169,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2040,6 +2195,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2259,7 +2416,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2284,6 +2441,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2302,6 +2462,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2350,6 +2511,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2373,6 +2535,14 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ ForgetParallelLocks();
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2385,9 +2555,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2405,7 +2580,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2426,7 +2604,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2478,6 +2656,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2531,6 +2713,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2566,11 +2749,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2852,6 +3037,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3241,6 +3427,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3418,6 +3605,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3511,6 +3708,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3540,6 +3747,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3560,6 +3779,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3594,6 +3814,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3617,6 +3849,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3694,6 +3927,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3718,6 +3963,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3806,6 +4052,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3828,6 +4088,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3860,6 +4121,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3882,6 +4155,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3893,6 +4174,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3968,6 +4250,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4059,6 +4342,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4162,6 +4446,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4315,6 +4606,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4455,6 +4753,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4502,6 +4801,134 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 3; /* top XID, current XID, count of XIDs */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * Currently, the only information we attempt to save and restore here is
+ * the XIDs associated with this transaction. The first eight bytes of the
+ * result contain the XID of the top-level transaction and the XID of the
+ * current transaction (or, in each case, InvalidTransactionId if none).
+ * The next 4 bytes contain a count of how many additional XIDs follow;
+ * this is followed by all of those XIDs one after another. We emit the XIDs
+ * in sorted order for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 3 * sizeof(TransactionId));
+ result[0] = XactTopTransactionId;
+ result[1] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 2) * sizeof(TransactionId));
+ result[2] = nParallelCurrentXids;
+ memcpy(&result[3], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[2] = (TransactionId) nxids;
+ memcpy(&result[3], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactTopTransactionId = tstate[0];
+ CurrentTransactionState->transactionId = tstate[1];
+ nParallelCurrentXids = (int) tstate[2];
+ ParallelCurrentXids = &tstate[3];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4571,6 +4998,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index bfb4fdc..76f43d8 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -3708,7 +3708,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3718,7 +3718,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 622ccf7..d3ccdb9 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -551,6 +551,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -838,6 +845,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 33b172b..07526e8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 84be37c..40f2eec7 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index 4b86e91..3a93a04 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index a1ebc72..32701d3 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1686,6 +1686,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..11a7978 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 28af40c..8899448 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2989,7 +2990,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 3533cfa..faa9b55 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -630,6 +650,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -640,6 +661,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -716,6 +738,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -747,6 +770,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 1b69322..affb23a 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -697,3 +698,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 9572777..de988ba 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5602,6 +5602,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6906,6 +6920,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..0685e64
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,63 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8205504..8fd3772 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index d2e5198..3112cd9 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..e00a459 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
On 2015-02-11 13:59:04 -0500, Robert Haas wrote:
On Tue, Feb 10, 2015 at 8:45 PM, Andres Freund <andres@2ndquadrant.com> wrote:
The only reason I'd like it to be active is because that'd *prohibit*
doing the crazier stuff. There seems little reason to not da it under
the additional protection against crazy things that'd give us?Trying to load additional libraries once parallel mode is in progress
can provide failures, because the load of the libraries causes the
system to do GUC_ACTION_SET on the GUCs whose initialization was
deferred, and that trips up the
no-changing-things-while-in-parallel-mode checks.
Oh, that's a good point.
I'm not sure if there's anything we can, or should, do about that.
Fine with me.
Well, ExportSnapshot()/Import has quite a bit more restrictions than
what you're doing... Most importantly it cannot import in transactions
unless using read committed isolation, forces xid assignment during
export, forces the old snapshot to stick around for the whole
transaction and only works on a primary. I'm not actually sure it makes
a relevant difference, but it's certainly worth calling attention to.The fuding I was wondering about certainly is unnecessary though -
GetSnapshotData() has already performed it...I'm not particularly awake right now and I think this needs a closer
look either by someone awake... I'm not fully convinced this is safe.I'm not 100% comfortable with it either, but I just spent some time
looking at it and can't see what's wrong with it. Basically, we're
trying to get the parallel worker into a state that matches the
master's state after doing GetTransactionSnapshot() - namely,
CurrentSnapshot should point to the same snapshot on the master, and
FirstSnapshotSet should be true, plus the same additional processing
that GetTransactionSnapshot() would have done if we're in a higher
transaction isolation level. It's possible we don't need to mimic
that state, but it seems like a good idea.
I plan to look at this soonish.
Still, I wonder if we ought to be banning GetTransactionSnapshot()
altogether. I'm not sure if there's ever a time when it's safe for a
worker to call that.
Why?
Also, you don't seem to
prohibit popping the active snapshot (should that be prohibitted
maybe?) which might bump the initiator's xmin horizon.I think as long as our transaction snapshot is installed correctly our
xmin horizon can't advance; am I missing something?Maybe I'm missing something, but why would that be the case in a read
committed session? ImportSnapshot() only ever calls
SetTransactionSnapshot it such a case (why does it contain code to cope
without it?), but your patch doesn't seem to guarantee that.But as don't actually transport XactIsoLevel anywhere (omission
probably?) that seems to mean that the SetTransactionSnapshot() won't do
much, even if the source transaction is repeatable read.Doesn't XactIsoLevel get set by assign_XactIsoLevel() when we restore
the GUC state?
Yes, but afaics the transaction start will just overwrite it again:
static void
StartTransaction(void)
{
...
XactDeferrable = DefaultXactDeferrable;
XactIsoLevel = DefaultXactIsoLevel;
...
For a client issued BEGIN it works because utility.c does:
case TRANS_STMT_BEGIN:
case TRANS_STMT_START:
{
ListCell *lc;
BeginTransactionBlock();
foreach(lc, stmt->options)
{
DefElem *item = (DefElem *) lfirst(lc);
if (strcmp(item->defname, "transaction_isolation") == 0)
SetPGVariable("transaction_isolation",
list_make1(item->arg),
true);
else if (strcmp(item->defname, "transaction_read_only") == 0)
SetPGVariable("transaction_read_only",
list_make1(item->arg),
true);
else if (strcmp(item->defname, "transaction_deferrable") == 0)
SetPGVariable("transaction_deferrable",
list_make1(item->arg),
true);
}
Pretty, isn't it?
Your manual ones don't either, that's what made me
complain. E.g. pg_advisory_lock_int8 isn't called that on the SQL
level. Instead they use the C name + parens.On further reflection, I think I should probably just change all of
those to use a general message about advisory locks, i.e.:ERROR: cannot use advisory locks during a parallel operation
That sound good to you?
Perfectly fine with me.
Which is *not* a good example for the problem. Your primary reasoning
for needing something more than sharing the locks that we know the
individual query is going to need (which we acquire in the parse
analzye/planner/executor) is that we can't predict what some random code
does. Now, I still don't think that argument should hold too much sway
because we'll only be able to run carefully controlled code
*anyway*.I'll avoid repeating what I've said about this before, except to say
that I still don't believe for a minute you can predict which locks
you'll need.
I don't understand. Leaving AEL locks on catalog tables aside, pretty
much everything else is easily visible? We already do that for RTE
permission checks and such? There might be some holes, but those should
rather be fixed anyway. What's so hard about determining the locks
required for a query?
But *if* we take it as reasoning for doing more than granting
the locks we need for the individual query, we *still* need to cope with
exactly the variants of the above invisible deadlock. You can still can
call a function acquiring some form of lock on either side.That by itself is not a deadlock. If the worker blocks trying to
acquire a lock that the master held before the start of parallelism,
it will wait forever, even if the master never acquires a lock. But
if the worker blocks trying to acquire a lock that the master acquired
*during* parallelism, it's presumably something like a catalog lock or
a tuple lock or a relation extension lock that the master is going to
release quickly.
If there's one lock that's acquired that way, there can easily be
two. And then they just need need to be acquired in an inconsistent
order and you have a deadlock.
There'll be many more of these, and we can't really
reason about them because we used to working locks.FUD.
It certainly scares me.
3. I welcome proposals for other ways of handling this problem, even
if they restrict the functionality that can be offered. For example,
a proposal that would make parallel_count revert to single-threaded
mode but terminate without an indefinite wait would be acceptable to
me, provided that it offers some advantage in safety and security over
what I've already got.I think the above example where we only grant the locks required for a
specific thing and only on the relevant severity would already be a big
improvement. Even better would be to to use the computed set of locks to
check whether workers could acquire them and refuse paralellism in that
case.I think that will just produce lots of very-hard-to-debug undetected
deadlocks and huge volumes of code that tries and fails to assess what
locks the worker will need.
Teaching the deadlock to recognize that
Another thing to do is to mark the lock, both in the master and workers,
as not effectively being of the original severity anymore. If the own
process again tries to acquire the lock on a heavier severity than what
was needed at the time of query execution, error out. At least until
parallel mode has confirmed to have ended. That should pretty much never
happen.I'm not sure what you mean by the "severity" of the lock.
The lock level.
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 Sun, Feb 15, 2015 at 11:48 AM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Feb 13, 2015 at 2:22 AM, Michael Paquier
<michael.paquier@gmail.com> wrote:On Thu, Feb 12, 2015 at 3:59 AM, Robert Haas <robertmhaas@gmail.com>
wrote:
We're not seeing eye to eye here yet, since I don't accept your
example scenario and you don't accept mine. Let's keep discussing.Meanwhile, here's an updated patch.
A lot of cool activity is showing up here, so moved the patch to CF
2015-02.
Perhaps Andres you could add yourself as a reviewer?
Here's a new version of the patch with (a) some additional
restrictions on heavyweight locking both at the start of, and during,
parallel mode and (b) a write-up in the README explaining the
restrictions and my theory of why the handling of heavyweight locking
is safe. Hopefully this goes some way towards addressing Andres's
concerns. I've also replaced the specific (and wrong) messages about
advisory locks with a more generic message, as previously proposed;
and I've fixed at least one bug.
Today, while testing parallel_seqscan patch, I encountered one
intermittent issue (it hangs in below function) and I have question
related to below function.
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
..
+ for (;;)
+ {
..
+ CHECK_FOR_INTERRUPTS();
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
Isn't there a race condition in this function such that after it finds
that there is some alive worker and before it does WaitLatch(), the
worker completes its work and exits, now in such a case who is
going to wake the backend waiting on procLatch?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On Fri, Mar 6, 2015 at 7:01 AM, Amit Kapila <amit.kapila16@gmail.com> wrote:
Today, while testing parallel_seqscan patch, I encountered one
intermittent issue (it hangs in below function) and I have question
related to below function.+void +WaitForParallelWorkersToFinish(ParallelContext *pcxt) +{ .. + for (;;) + { .. + CHECK_FOR_INTERRUPTS(); + for (i = 0; i < pcxt->nworkers; ++i) + { + if (pcxt->worker[i].error_mqh != NULL) + { + anyone_alive = true; + break; + } + } + + if (!anyone_alive) + break; + + WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1); + ResetLatch(&MyProc->procLatch); + }Isn't there a race condition in this function such that after it finds
that there is some alive worker and before it does WaitLatch(), the
worker completes its work and exits, now in such a case who is
going to wake the backend waiting on procLatch?
It doesn't matter whether some other backend sets the process latch
before we reach WaitLatch() or after we begin waiting. Either way,
it's fine.
It would be bad if the process could exit without setting the latch at
all, though. I hope that's not the case.
--
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, Feb 17, 2015 at 11:01 AM, Andres Freund <andres@2ndquadrant.com> wrote:
I'm not 100% comfortable with it either, but I just spent some time
looking at it and can't see what's wrong with it. Basically, we're
trying to get the parallel worker into a state that matches the
master's state after doing GetTransactionSnapshot() - namely,
CurrentSnapshot should point to the same snapshot on the master, and
FirstSnapshotSet should be true, plus the same additional processing
that GetTransactionSnapshot() would have done if we're in a higher
transaction isolation level. It's possible we don't need to mimic
that state, but it seems like a good idea.I plan to look at this soonish.
Did you get a chance to look at this yet?
Still, I wonder if we ought to be banning GetTransactionSnapshot()
altogether. I'm not sure if there's ever a time when it's safe for a
worker to call that.Why?
I don't know. I looked at it more and I don't really see a problem,
but what do I know?
Doesn't XactIsoLevel get set by assign_XactIsoLevel() when we restore
the GUC state?Yes, but afaics the transaction start will just overwrite it again:
static void
StartTransaction(void)
{
...
XactDeferrable = DefaultXactDeferrable;
XactIsoLevel = DefaultXactIsoLevel;
...
Ah, crap. So, yeah, we need to save and restore that, too. Fixed in
the attached version.
For a client issued BEGIN it works because utility.c does:
case TRANS_STMT_BEGIN:
case TRANS_STMT_START:
{
ListCell *lc;BeginTransactionBlock();
foreach(lc, stmt->options)
{
DefElem *item = (DefElem *) lfirst(lc);if (strcmp(item->defname, "transaction_isolation") == 0)
SetPGVariable("transaction_isolation",
list_make1(item->arg),
true);
else if (strcmp(item->defname, "transaction_read_only") == 0)
SetPGVariable("transaction_read_only",
list_make1(item->arg),
true);
else if (strcmp(item->defname, "transaction_deferrable") == 0)
SetPGVariable("transaction_deferrable",
list_make1(item->arg),
true);
}Pretty, isn't it?
I think that's just to handle things like BEGIN ISOLATION LEVEL
SERIALIZABLE. A plain BEGIN would work fine without that AFAICS. It
doesn't seem particularly ugly to me either, but I guess that's
neither here nor there.
I'll avoid repeating what I've said about this before, except to say
that I still don't believe for a minute you can predict which locks
you'll need.I don't understand. Leaving AEL locks on catalog tables aside, pretty
much everything else is easily visible? We already do that for RTE
permission checks and such? There might be some holes, but those should
rather be fixed anyway. What's so hard about determining the locks
required for a query?
Well, if you're only going to call built-in functions, and if you
exclude all of the built-in functions that that can take locks on
arbitrary objects like pg_get_object_address() and table_to_xml(), and
if you leave locks on catalog tables aside, then, given further that
we're restricting ourselves to read-only transactions, you *can*
determine the locks that will be required for a query -- there won't
be any. But one cannot exclude catalog tables by fiat, and all of
those other restrictions are ones that I'd like to have a chance of
relaxing at some point. It's entirely reasonable for a user to want
to parallelize a query that contains a user-defined PL/pgsql function,
though, and that might do anything.
If there's one lock that's acquired that way, there can easily be
two. And then they just need need to be acquired in an inconsistent
order and you have a deadlock.
There is a detailed and hopefully rigorous analysis of locking-related
scenarios in README.parallel in the patch version after the one your
reviewed (posted 2015-02-15). Have you looked at that? (It's also in
this version.)
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v7.patchbinary/octet-stream; name=parallel-mode-v7.patchDownload
From 803e8ef89e9719bae742852e60cff3d6a4ef5d77 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/4] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Simon
Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 42 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 276 +++++++++
src/backend/access/transam/parallel.c | 922 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 464 +++++++++++++-
src/backend/catalog/namespace.c | 4 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 ++++++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 +++
src/backend/utils/time/snapmgr.c | 210 ++++++-
src/include/access/parallel.h | 63 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 10 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
39 files changed, 2656 insertions(+), 52 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb6f8a3..173f0ba 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..720f3d8
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,276 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated parallelism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+ - Locks on relations and database objects, and any lock held on the transaction's
+ XID. See lock management, below.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the master, the
+parallel backend ends up with a stack of depth 1. This stack entry is
+marked with the special transaction block state TBLOCK_PARALLEL_INPROGRESS
+so that it's not confused with an ordinary toplevel transaction. The
+XID of this TransactionState is set to the XID of the innermost
+currently-active subtransaction in the initiating backend. The initiating
+backend's toplevel XID, and the XIDs of all current (in-progress or
+subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(),
+and TransactionIdIsCurrentTransactionId() return the same values that they
+would in the initiating backend. We could copy the entire transaction state
+stack, but most of it would be useless: for example, you can't roll back to
+a savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initiating
+backend to exit any transaction or subtransaction that was in progress when
+parallelism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context that was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+Transaction commit or abort requires careful coordination between backends.
+Each backend has its own resource owners: buffer pins, catcache or relcache
+reference counts, tuple descriptors, and so on are managed separately by each
+backend, and each backend is separately responsible for releasing such
+resources. Generally, the commit or abort of a parallel worker is much like
+a top-transaction commit or abort, but there are a few exceptions. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is
+ responsible for this, too.
+
+The master kills off all remaining workers as part of commit or abort
+processing. It must not only kill the workers but wait for them to actually
+exit; otherwise, chaos can ensue. For example, if the master is
+rolling back the transaction that created the relation being scanned by
+a worker, the relation could disappear while the worker is still busy
+scanning it. That's not safe.
+
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+master holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and perhaps the lock
+on our transaction ID are copied to workers. Advisory locks are not copied,
+but the master may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe to
+attempt parallelism while holding a lock of any other type, such as a page,
+tuple, or relation extension lock, and such attempts will fail. Although
+there is currently no reason to do so, such locks could be taken and released
+during parallel mode; they merely cannot be held at the start of parallel
+mode, since we would then fail to provide necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the parallel processe and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..8d39bf2
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,922 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ CheckForRetainedParallelLocks();
+
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 89769ea..7b3a691 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,51 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+
+ if (s->parallelModeLevel == 0)
+ CheckForRetainedParallelLocks();
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +927,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1710,6 +1843,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1839,6 +1974,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1872,7 +2010,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1881,6 +2020,14 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ CheckForRetainedParallelLocks();
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1920,9 +2067,13 @@ CommitTransaction(void)
s->state = TRANS_COMMIT;
/*
- * Here is where we really truly commit.
+ * Unless we're in parallel mode, we need to mark our XIDs as committed
+ * in pg_clog. This is where durably commit.
*/
- latestXid = RecordTransactionCommit();
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionCommit();
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1949,7 +2100,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1997,7 +2149,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2022,6 +2174,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2045,6 +2200,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2264,7 +2421,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2289,6 +2446,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2307,6 +2467,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2355,6 +2516,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2378,6 +2540,14 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ ForgetParallelLocks();
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2390,9 +2560,14 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (parallel)
+ latestXid = InvalidTransactionId;
+ else
+ latestXid = RecordTransactionAbort(false);
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2410,7 +2585,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2431,7 +2609,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2483,6 +2661,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2536,6 +2718,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2571,11 +2754,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2857,6 +3042,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3246,6 +3432,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3423,6 +3610,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3516,6 +3713,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3545,6 +3752,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3565,6 +3784,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3599,6 +3819,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3622,6 +3854,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3699,6 +3932,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3723,6 +3968,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3811,6 +4057,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3833,6 +4093,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3865,6 +4126,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3887,6 +4160,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3898,6 +4179,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3973,6 +4255,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4064,6 +4347,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4167,6 +4451,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4320,6 +4611,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4460,6 +4758,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4507,6 +4806,139 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 5; /* iso level, deferrable, top & current XID, XID count */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * We need to save and restore XactDeferrable, XactIsoLevel, and the XIDs
+ * associated with this transaction. The first eight bytes of the result
+ * contain XactDeferrable and XactIsoLevel; the next eight bytes contain the
+ * XID of the top-level transaction and the XID of the current transaction
+ * (or, in each case, InvalidTransactionId if none). After that, the next 4
+ * bytes contain a count of how many additional XIDs follow; this is followed
+ * by all of those XIDs one after another. We emit the XIDs in sorted order
+ * for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 5 * sizeof(TransactionId));
+ result[0] = (TransactionId) XactIsoLevel;
+ result[1] = (TransactionId) XactDeferrable;
+ result[2] = XactTopTransactionId;
+ result[3] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 4) * sizeof(TransactionId));
+ result[4] = nParallelCurrentXids;
+ memcpy(&result[5], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[4] = (TransactionId) nxids;
+ memcpy(&result[5], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactIsoLevel = (int) tstate[0];
+ XactDeferrable = (bool) tstate[1];
+ XactTopTransactionId = tstate[2];
+ CurrentTransactionState->transactionId = tstate[3];
+ nParallelCurrentXids = (int) tstate[4];
+ ParallelCurrentXids = &tstate[5];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4576,6 +5008,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 1af977c..2e4fdd4 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -3709,7 +3709,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3719,7 +3719,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 6d316d6..80f5553 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -565,6 +565,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -862,6 +869,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 33b172b..07526e8 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 6c3eff7..ce49c47 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index b3c0502..557d153 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 8eaec0c..68cc6ed 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1683,6 +1683,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..b19bffa 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 33720e8..ea2a432 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2988,7 +2989,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index daf5326..16bb594 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -617,6 +637,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -627,6 +648,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -703,6 +725,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -734,6 +757,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 7476a26..46bc1f2 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -692,3 +693,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index d84dba7..791543e 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5651,6 +5651,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6955,6 +6969,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..0685e64
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,63 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index d7e5f64..608eb21 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index cf5f7d0..f3b005f 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1100923..e00a459 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
On Fri, Mar 6, 2015 at 10:16 AM, Robert Haas <robertmhaas@gmail.com> wrote:
[ responses to review comments ]
Here's a new version, fixing a bug Amit found (missing SetLatch) and a
more significant oversight identified by Noah (XactLastRecEnd
propagation) on the "assessing parallel safety" thread.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v8.patchbinary/octet-stream; name=parallel-mode-v8.patchDownload
From 6aa1b0917e0bc88fe0d6140e623744e6879448a8 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/5] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 42 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 283 ++++++++
src/backend/access/transam/parallel.c | 962 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 491 +++++++++++++-
src/backend/access/transam/xlog.c | 8 +
src/backend/catalog/namespace.c | 4 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 ++++++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 +++
src/backend/utils/time/snapmgr.c | 210 +++++-
src/include/access/parallel.h | 65 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 10 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
40 files changed, 2738 insertions(+), 54 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb6f8a3..173f0ba 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..74b00de
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,283 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated parallelism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+ - Locks on relations and database objects, and any lock held on the
+ transaction's XID. See lock management, below.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the parallel
+leader, each parallel worker ends up with a stack of depth 1. This stack
+entry is marked with the special transaction block state
+TBLOCK_PARALLEL_INPROGRESS so that it's not confused with an ordinary
+toplevel transaction. The XID of this TransactionState is set to the XID of
+the innermost currently-active subtransaction in the initiating backend. The
+initiating backend's toplevel XID, and the XIDs of all current (in-progress
+or subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(), and
+TransactionIdIsCurrentTransactionId() return the same values that they would
+in the initiating backend. We could copy the entire transaction state stack,
+but most of it would be useless: for example, you can't roll back to a
+savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initiating
+backend to exit any transaction or subtransaction that was in progress when
+parallelism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context than was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+At the end of a parallel operation, which can happen either because it
+completed successfully or because it was interrupted by an error, parallel
+workers associated with that operation exit. In the error case, transaction
+abort processing in the parallel leader kills of any remaining workers, and
+the parallel leader then waits for them to die. In the case of a successful
+parallel operation, the parallel leader does not send any signals, but must
+wait for workers to complete and exit of their own volition. In either
+case, it is very important that all workers actually exit before the
+parallel leader cleans up the (sub)transaction in which they were created;
+otherwise, chaos can ensue. For example, if the leader is rolling back the
+transaction that created the relation being scanned by a worker, the
+relation could disappear while the worker is still busy scanning it. That's
+not safe.
+
+Generally, the cleanup performed by each worker at this point is similar to
+top-level commit or abort. Each backend has its own resource owners: buffer
+pins, catcache or relcache reference counts, tuple descriptors, and so on
+are managed separately by each backend, and must free them before exiting.
+There are, however, some important differences between parallel worker
+commit or abort and a real top-level transaction commit or abort. Most
+imporantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is
+ responsible for this, too.
+
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+leader holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and perhaps the lock
+on our transaction ID are copied to workers. Advisory locks are not copied,
+but the leader may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe to
+attempt parallelism while holding a lock of any other type, such as a page,
+tuple, or relation extension lock, and such attempts will fail. Although
+there is currently no reason to do so, such locks could be taken and released
+during parallel mode; they merely cannot be held at the start of parallel
+mode, since we would then fail to provide necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the parallel processe and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..de56476
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,962 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/xlog.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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;
+
+ /* Mutex protects remaining fields. */
+ slock_t mutex;
+
+ /* Track whether workers have attached. */
+ int workers_expected;
+ int workers_attached;
+
+ /* Maximum XactLastRecEnd of any worker. */
+ XLogRecPtr last_xlog_end;
+} 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Pointer to our fixed parallel state. */
+static FixedParallelState *MyFixedParallelState;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ fps->last_xlog_end = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ *
+ * Also, we want to update our notion of XactLastRecEnd based on worker
+ * feedback.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ CheckForRetainedParallelLocks();
+
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ if (pcxt->toc != NULL)
+ {
+ FixedParallelState *fps;
+
+ fps = shm_toc_lookup(pcxt->toc, PARALLEL_KEY_FIXED);
+ if (fps->last_xlog_end > XactLastRecEnd)
+ XactLastRecEnd = fps->last_xlog_end;
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+ SetLatch(MyLatch);
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+ MyFixedParallelState = fps;
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
+
+/*
+ * Update shared memory with the ending location of the last WAL record we
+ * wrote, if it's greater than the value already stored there.
+ */
+void
+ParallelWorkerReportLastRecEnd(XLogRecPtr last_xlog_end)
+{
+ FixedParallelState *fps = MyFixedParallelState;
+
+ Assert(fps != NULL);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->last_xlog_end < last_xlog_end)
+ fps->last_xlog_end = last_xlog_end;
+ SpinLockRelease(&fps->mutex);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 89769ea..e192f57 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,51 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+
+ if (s->parallelModeLevel == 0)
+ CheckForRetainedParallelLocks();
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +927,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1710,6 +1843,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1839,6 +1974,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1872,7 +2010,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1881,6 +2020,14 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ CheckForRetainedParallelLocks();
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1919,10 +2066,28 @@ CommitTransaction(void)
*/
s->state = TRANS_COMMIT;
- /*
- * Here is where we really truly commit.
- */
- latestXid = RecordTransactionCommit();
+ if (!parallel)
+ {
+ /*
+ * We need to mark our XIDs as commited in pg_clog. This is where we
+ * durably commit.
+ */
+ latestXid = RecordTransactionCommit();
+ }
+ else
+ {
+ /*
+ * We must not mark our XID committed; the parallel master is
+ * responsible for that.
+ */
+ latestXid = InvalidTransactionId;
+
+ /*
+ * Make sure the master will know about any WAL we wrote before it
+ * commits.
+ */
+ ParallelWorkerReportLastRecEnd(XactLastRecEnd);
+ }
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1949,7 +2114,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1997,7 +2163,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2022,6 +2188,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -2045,6 +2214,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2264,7 +2435,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2289,6 +2460,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2307,6 +2481,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2355,6 +2530,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2378,6 +2554,14 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ ForgetParallelLocks();
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2390,9 +2574,23 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ 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);
+ }
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2410,7 +2608,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2431,7 +2632,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2483,6 +2684,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2536,6 +2741,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2571,11 +2777,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2857,6 +3065,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3246,6 +3455,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3423,6 +3633,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3516,6 +3736,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3545,6 +3775,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3565,6 +3807,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3599,6 +3842,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3622,6 +3877,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3699,6 +3955,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3723,6 +3991,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3811,6 +4080,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3833,6 +4116,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3865,6 +4149,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3887,6 +4183,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3898,6 +4202,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3973,6 +4278,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -4064,6 +4370,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4167,6 +4474,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4320,6 +4634,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4460,6 +4781,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4507,6 +4829,139 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 5; /* iso level, deferrable, top & current XID, XID count */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * We need to save and restore XactDeferrable, XactIsoLevel, and the XIDs
+ * associated with this transaction. The first eight bytes of the result
+ * contain XactDeferrable and XactIsoLevel; the next eight bytes contain the
+ * XID of the top-level transaction and the XID of the current transaction
+ * (or, in each case, InvalidTransactionId if none). After that, the next 4
+ * bytes contain a count of how many additional XIDs follow; this is followed
+ * by all of those XIDs one after another. We emit the XIDs in sorted order
+ * for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 5 * sizeof(TransactionId));
+ result[0] = (TransactionId) XactIsoLevel;
+ result[1] = (TransactionId) XactDeferrable;
+ result[2] = XactTopTransactionId;
+ result[3] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 4) * sizeof(TransactionId));
+ result[4] = nParallelCurrentXids;
+ memcpy(&result[5], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[4] = (TransactionId) nxids;
+ memcpy(&result[5], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactIsoLevel = (int) tstate[0];
+ XactDeferrable = (bool) tstate[1];
+ XactTopTransactionId = tstate[2];
+ CurrentTransactionState->transactionId = tstate[3];
+ nParallelCurrentXids = (int) tstate[4];
+ ParallelCurrentXids = &tstate[5];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4576,6 +5031,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 554491b..acc5ffe 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -291,6 +291,14 @@ static TimeLineID curFileTLI;
* end+1 of the last record, and is reset when we end a top-level transaction,
* or start a new one; so it can be used to tell if the current transaction has
* created any XLOG records.
+ *
+ * While in parallel mode, this may not be fully up to date. When committing,
+ * a transaction can assume this covers all xlog records written either by the
+ * user backend or by any parallel worker which was present at any point during
+ * the transaction. But when aborting, or when still in parallel mode, other
+ * parallel backends may have written WAL records at later LSNs than the value
+ * stored here. The parallel leader advances its own copy, when necessary,
+ * in WaitForParallelWorkersToFinish.
*/
static XLogRecPtr ProcLastRecPtr = InvalidXLogRecPtr;
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 1af977c..2e4fdd4 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -3709,7 +3709,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3719,7 +3719,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 6d316d6..80f5553 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -565,6 +565,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -862,6 +869,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 13ceffa..3cf94ff 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 6c3eff7..ce49c47 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index b3c0502..557d153 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 8eaec0c..68cc6ed 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1683,6 +1683,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..b19bffa 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 33720e8..ea2a432 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2988,7 +2989,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 126e38d..c4dabb1 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -617,6 +637,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -627,6 +648,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -703,6 +725,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -734,6 +757,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 7476a26..46bc1f2 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -692,3 +693,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 7196b0b..7ea4209 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5671,6 +5671,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6975,6 +6989,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..9c68907
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,65 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "access/xlogdefs.h"
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+extern void ParallelWorkerReportLastRecEnd(XLogRecPtr);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index d7e5f64..608eb21 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -77,9 +77,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -241,6 +244,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -260,4 +267,8 @@ extern void xact_redo(XLogReaderState *record);
extern void xact_desc(StringInfo buf, XLogReaderState *record);
extern const char *xact_identify(uint8 info);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index cf5f7d0..f3b005f 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1477a6f..bd3e90d 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
On Tue, Mar 17, 2015 at 6:53 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Fri, Mar 6, 2015 at 10:16 AM, Robert Haas <robertmhaas@gmail.com> wrote:
[ responses to review comments ]
Here's a new version, fixing a bug Amit found (missing SetLatch) and a
more significant oversight identified by Noah (XactLastRecEnd
propagation) on the "assessing parallel safety" thread.
Rebased.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v8.1.patchbinary/octet-stream; name=parallel-mode-v8.1.patchDownload
From dd5f6a7a642814c3e034e0eae6f9cdecaa214311 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/5] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 42 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 283 ++++++++
src/backend/access/transam/parallel.c | 962 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 491 +++++++++++++-
src/backend/access/transam/xlog.c | 8 +
src/backend/catalog/namespace.c | 4 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 ++++++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 +++
src/backend/utils/time/snapmgr.c | 210 +++++-
src/include/access/parallel.h | 65 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 10 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
40 files changed, 2738 insertions(+), 54 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb6f8a3..173f0ba 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2234,6 +2234,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2652,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3100,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5413,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..74b00de
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,283 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated parallelism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach: we try to make as many of the
+operations that are safe outside of parallel mode work correctly in parallel
+mode as well, and we try to prohibit the rest via suitable error checks.
+The error checks are engaged via EnterParallelMode(), which should be called
+before creating a parallel context, and disarmed via ExitParallelMode(),
+which should be called after all parallel contexts have been destroyed.
+The most significant restriction imposed by parallel mode is that all
+operations must be strictly read-only; we allow no writes to the database
+and no DDL. We might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are potentially OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+ - Locks on relations and database objects, and any lock held on the
+ transaction's XID. See lock management, below.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the parallel
+leader, each parallel worker ends up with a stack of depth 1. This stack
+entry is marked with the special transaction block state
+TBLOCK_PARALLEL_INPROGRESS so that it's not confused with an ordinary
+toplevel transaction. The XID of this TransactionState is set to the XID of
+the innermost currently-active subtransaction in the initiating backend. The
+initiating backend's toplevel XID, and the XIDs of all current (in-progress
+or subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(), and
+TransactionIdIsCurrentTransactionId() return the same values that they would
+in the initiating backend. We could copy the entire transaction state stack,
+but most of it would be useless: for example, you can't roll back to a
+savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initiating
+backend to exit any transaction or subtransaction that was in progress when
+parallelism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context than was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+At the end of a parallel operation, which can happen either because it
+completed successfully or because it was interrupted by an error, parallel
+workers associated with that operation exit. In the error case, transaction
+abort processing in the parallel leader kills of any remaining workers, and
+the parallel leader then waits for them to die. In the case of a successful
+parallel operation, the parallel leader does not send any signals, but must
+wait for workers to complete and exit of their own volition. In either
+case, it is very important that all workers actually exit before the
+parallel leader cleans up the (sub)transaction in which they were created;
+otherwise, chaos can ensue. For example, if the leader is rolling back the
+transaction that created the relation being scanned by a worker, the
+relation could disappear while the worker is still busy scanning it. That's
+not safe.
+
+Generally, the cleanup performed by each worker at this point is similar to
+top-level commit or abort. Each backend has its own resource owners: buffer
+pins, catcache or relcache reference counts, tuple descriptors, and so on
+are managed separately by each backend, and must free them before exiting.
+There are, however, some important differences between parallel worker
+commit or abort and a real top-level transaction commit or abort. Most
+imporantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is
+ responsible for this, too.
+
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+leader holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and perhaps the lock
+on our transaction ID are copied to workers. Advisory locks are not copied,
+but the leader may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe to
+attempt parallelism while holding a lock of any other type, such as a page,
+tuple, or relation extension lock, and such attempts will fail. Although
+there is currently no reason to do so, such locks could be taken and released
+during parallel mode; they merely cannot be held at the start of parallel
+mode, since we would then fail to provide necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the parallel processe and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..de56476
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,962 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/xlog.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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;
+
+ /* Mutex protects remaining fields. */
+ slock_t mutex;
+
+ /* Track whether workers have attached. */
+ int workers_expected;
+ int workers_attached;
+
+ /* Maximum XactLastRecEnd of any worker. */
+ XLogRecPtr last_xlog_end;
+} 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Pointer to our fixed parallel state. */
+static FixedParallelState *MyFixedParallelState;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be positive. */
+ Assert(nworkers >= 0);
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len;
+ Size guc_len;
+ Size combocidlen;
+ Size tsnaplen;
+ Size asnaplen;
+ Size tstatelen;
+ Size lockstatelen;
+ Size segsize;
+ int i;
+ FixedParallelState *fps;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Estimate how much space we'll need for state sharing.
+ *
+ * If you add more chunks here, you probably need more keys, too.
+ */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ shm_toc_estimate_keys(&pcxt->estimator, 8);
+
+ /* Estimate how much space we'll need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+
+ /* Create DSM and initialize with new table of contents. */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ pcxt->seg = dsm_create(segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ fps->last_xlog_end = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC state to dynamic shared memory. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state to dynamic shared memory. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshots to dynamic shared memory. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT, tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state to dynamic shared memory. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize lock state to dynamic shared memory. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data should
+ * be transmitted via separate (possibly larger?) queue.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc, PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ shm_mq *mq;
+
+ mq = shm_mq_create(error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE,
+ PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information to dynamic shared memory. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ *
+ * Also, we want to update our notion of XactLastRecEnd based on worker
+ * feedback.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ CheckForRetainedParallelLocks();
+
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ if (pcxt->toc != NULL)
+ {
+ FixedParallelState *fps;
+
+ fps = shm_toc_lookup(pcxt->toc, PARALLEL_KEY_FIXED);
+ if (fps->last_xlog_end > XactLastRecEnd)
+ XactLastRecEnd = fps->last_xlog_end;
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ dsm_detach(pcxt->seg);
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+ SetLatch(MyLatch);
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+ MyFixedParallelState = fps;
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
+
+/*
+ * Update shared memory with the ending location of the last WAL record we
+ * wrote, if it's greater than the value already stored there.
+ */
+void
+ParallelWorkerReportLastRecEnd(XLogRecPtr last_xlog_end)
+{
+ FixedParallelState *fps = MyFixedParallelState;
+
+ Assert(fps != NULL);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->last_xlog_end < last_xlog_end)
+ fps->last_xlog_end = last_xlog_end;
+ SpinLockRelease(&fps->mutex);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1495bb4..4969231 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,51 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+
+ if (s->parallelModeLevel == 0)
+ CheckForRetainedParallelLocks();
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +927,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1640,6 +1773,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1769,6 +1904,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1802,7 +1940,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1811,6 +1950,14 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ CheckForRetainedParallelLocks();
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1849,10 +1996,28 @@ CommitTransaction(void)
*/
s->state = TRANS_COMMIT;
- /*
- * Here is where we really truly commit.
- */
- latestXid = RecordTransactionCommit();
+ if (!parallel)
+ {
+ /*
+ * We need to mark our XIDs as commited in pg_clog. This is where we
+ * durably commit.
+ */
+ latestXid = RecordTransactionCommit();
+ }
+ else
+ {
+ /*
+ * We must not mark our XID committed; the parallel master is
+ * responsible for that.
+ */
+ latestXid = InvalidTransactionId;
+
+ /*
+ * Make sure the master will know about any WAL we wrote before it
+ * commits.
+ */
+ ParallelWorkerReportLastRecEnd(XactLastRecEnd);
+ }
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1879,7 +2044,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1927,7 +2093,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -1952,6 +2118,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -1975,6 +2144,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2194,7 +2365,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2219,6 +2390,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2237,6 +2411,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2285,6 +2460,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2308,6 +2484,14 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ ForgetParallelLocks();
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2320,9 +2504,23 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ 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);
+ }
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2340,7 +2538,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2361,7 +2562,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2413,6 +2614,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2466,6 +2671,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2501,11 +2707,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2787,6 +2995,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3176,6 +3385,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3353,6 +3563,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3446,6 +3666,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3475,6 +3705,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3495,6 +3737,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3529,6 +3772,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3552,6 +3807,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3629,6 +3885,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3653,6 +3921,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3741,6 +4010,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3763,6 +4046,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3795,6 +4079,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3817,6 +4113,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3828,6 +4132,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3903,6 +4208,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -3994,6 +4300,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4097,6 +4404,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4250,6 +4564,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4390,6 +4711,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4437,6 +4759,139 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 5; /* iso level, deferrable, top & current XID, XID count */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * We need to save and restore XactDeferrable, XactIsoLevel, and the XIDs
+ * associated with this transaction. The first eight bytes of the result
+ * contain XactDeferrable and XactIsoLevel; the next eight bytes contain the
+ * XID of the top-level transaction and the XID of the current transaction
+ * (or, in each case, InvalidTransactionId if none). After that, the next 4
+ * bytes contain a count of how many additional XIDs follow; this is followed
+ * by all of those XIDs one after another. We emit the XIDs in sorted order
+ * for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 5 * sizeof(TransactionId));
+ result[0] = (TransactionId) XactIsoLevel;
+ result[1] = (TransactionId) XactDeferrable;
+ result[2] = XactTopTransactionId;
+ result[3] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 4) * sizeof(TransactionId));
+ result[4] = nParallelCurrentXids;
+ memcpy(&result[5], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[4] = (TransactionId) nxids;
+ memcpy(&result[5], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactIsoLevel = (int) tstate[0];
+ XactDeferrable = (bool) tstate[1];
+ XactTopTransactionId = tstate[2];
+ CurrentTransactionState->transactionId = tstate[3];
+ nParallelCurrentXids = (int) tstate[4];
+ ParallelCurrentXids = &tstate[5];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4506,6 +4961,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index e2d187f..57ba1a3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -291,6 +291,14 @@ static TimeLineID curFileTLI;
* end+1 of the last record, and is reset when we end a top-level transaction,
* or start a new one; so it can be used to tell if the current transaction has
* created any XLOG records.
+ *
+ * While in parallel mode, this may not be fully up to date. When committing,
+ * a transaction can assume this covers all xlog records written either by the
+ * user backend or by any parallel worker which was present at any point during
+ * the transaction. But when aborting, or when still in parallel mode, other
+ * parallel backends may have written WAL records at later LSNs than the value
+ * stored here. The parallel leader advances its own copy, when necessary,
+ * in WaitForParallelWorkersToFinish.
*/
static XLogRecPtr ProcLastRecPtr = InvalidXLogRecPtr;
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 1af977c..2e4fdd4 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -3709,7 +3709,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3719,7 +3719,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 6d316d6..80f5553 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -565,6 +565,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -862,6 +869,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 13ceffa..3cf94ff 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 6c3eff7..ce49c47 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index b3c0502..557d153 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 267b916..f80141a 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -966,6 +966,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 8eaec0c..68cc6ed 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1683,6 +1683,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..b19bffa 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 33720e8..ea2a432 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2988,7 +2989,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 065475d..d6097e8 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -617,6 +637,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -627,6 +648,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
vacuum(stmt, InvalidOid, true, NULL, false, isTopLevel);
}
break;
@@ -703,6 +725,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -734,6 +757,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 7476a26..46bc1f2 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -692,3 +693,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 26275bd..9c74ed3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5671,6 +5671,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6975,6 +6989,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..9c68907
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,65 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "access/xlogdefs.h"
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+extern void ParallelWorkerReportLastRecEnd(XLogRecPtr);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index fdf3ea3..60ed01a 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -78,9 +78,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -321,6 +324,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -357,4 +364,8 @@ extern const char *xact_identify(uint8 info);
extern void ParseCommitRecord(uint8 info, xl_xact_commit *xlrec, xl_xact_parsed_commit *parsed);
extern void ParseAbortRecord(uint8 info, xl_xact_abort *xlrec, xl_xact_parsed_abort *parsed);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index cf5f7d0..f3b005f 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1477a6f..bd3e90d 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
Hi,
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!
On 2015-03-18 12:02:07 -0400, Robert Haas wrote:
+Instead, we take a more pragmatic approach: we try to make as many of the +operations that are safe outside of parallel mode work correctly in parallel +mode as well, and we try to prohibit the rest via suitable error checks.
I'd say that we'd try to prohibit a bunch of common cases. Among them
the ones that are triggerable from SQL. We don't really try to prohibit
many kinds of traps, as you describe.
+ - The authenticated user ID and current database. Each parallel worker + will connect to the same database as the initiating backend, using the + same user ID.
This sentence immediately makes me wonder why only the authenticated
user, and not the currently active role, is mentioned.
+ - The values of all GUCs. Accordingly, permanent changes to the value of + any GUC are forbidden while in parallel mode; but temporary changes, + such as entering a function with non-NULL proconfig, are potentially OK.
"potentially OK" sounds odd to me. Which danger are you seing that isn't
relevan tfor norm
+ - The combo CID mappings. This is needed to ensure consistent answers to + tuple visibility checks. The need to synchronize this data structure is + a major reason why we can't support writes in parallel mode: such writes + might create new combo CIDs, and we have now way to let other workers + (or the initiating backend) know about them.
Absolutely *not* in the initial version, but we really need to find a
better solution for this.
+ - The currently active user ID and security context. Note that this is + the fourth user ID we restore: the initial step of binding to the correct + database also involves restoring the authenticated user ID. When GUC + values are restored, this incidentally sets SessionUserId and OuterUserId + to the correct values. This final step restores CurrentUserId.
Ah. That's the answer for above. Could you just move it next to the
other user bit?
+We could copy the entire transaction state stack, +but most of it would be useless: for example, you can't roll back to a +savepoint from within a parallel worker, and there are no resources to +associated with the memory contexts or resource owners of intermediate +subtransactions.
I do wonder if we're not going to have to change this in the not too far
away future. But then, this isn't externally visible at all, so
whatever.
+At the end of a parallel operation, which can happen either because it +completed successfully or because it was interrupted by an error, parallel +workers associated with that operation exit. In the error case, transaction +abort processing in the parallel leader kills of any remaining workers, and +the parallel leader then waits for them to die.
Very slightly awkward because first you talk about successful *or* error
and then about abort processing.
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is + responsible for this, too.
How could a worker have its own pg_temp namespace?
+Lock Management +=============== + +Certain heavyweight locks that the initiating backend holds at the beginning +of parallelism are copied to each worker, which unconditionally acquires them. +The parallel backends acquire - without waiting - each such lock that the +leader holds, even if that lock is self-exclusive. This creates the unusual +situation that a lock which could normally only be held by a single backend +can be shared among several backends in a parallel group. + +Obviously, this presents significant hazards that would not be present in +normal execution. If, for example, a backend were to initiate parallelism +while ReindexIsProcessingIndex() were true for some index, the parallel +backends launched at that time would neither share this state nor be excluded +from accessing the index via the heavyweight lock mechanism. It is therefore +imperative that backends only initiate parallelism from places where it will +be safe for parallel workers to access the relations on which they hold locks. +It is also important that they not afterwards do anything which causes access +to those relations to become unsafe, or at least not until after parallelism +has concluded. The fact that parallelism is strictly read-only means that the +opportunities for such mishaps are few and far between; furthermore, most +operations which present these hazards are DDL operations, which will be +rejected by CheckTableNotInUse() if parallel mode is active. + +Only relation locks, locks on database or shared objects, and perhaps the lock +on our transaction ID are copied to workers.
"perhaps"?
Advisory locks are not copied, +but the leader may hold them at the start of parallelism; they cannot +subsequently be manipulated while parallel mode is active. It is not safe to +attempt parallelism while holding a lock of any other type, such as a page, +tuple, or relation extension lock, and such attempts will fail. Although +there is currently no reason to do so, such locks could be taken and released +during parallel mode; they merely cannot be held at the start of parallel +mode, since we would then fail to provide necessary mutual exclusion.
Is it really true that no such locks are acquired? What about e.g. hash
indexes? They seem to be acquiring page locks while searching.
+Copying locks to workers is important for the avoidance of undetected +deadlock between the initiating process and its parallel workers. If the +initiating process holds a lock on an object at the start of parallelism, +and the worker subsequently attempts to acquire a lock on that same object +and blocks, this will result in an undetected deadlock, because the +initiating process cannot finish the transaction (thus releasing the lock) +until the worker terminates, and the worker cannot acquire the lock while +the initiating process holds it. Locks which the processes involved acquire +and then release during parallelism do not present this hazard; they simply +force the processes involved to take turns accessing the protected resource.
I don't think this is a strong guarantee. There very well can be lack of
forward progress if they're waiting on each other in some way. Say the
master backend holds the lock and waits on output from a worker. The
worker then will endlessly wait for the lock to become free. A
deadlock. Or, as another scenario, consider cooperating backends that
both try to send tuples to each other but the queue is full. A deadlock.
To me it seems the deadlock detector has to be enhanced to be able to
see 'waiting for' edges. Independent on how we resolve our difference of
opinion on the copying of locks.
It seems to me that this isn't all that hard: Whenever we block waiting
for another backend we enter ourselves on the wait queue to that
backend's virtual transaction. When finished we take the blocking
backend off. That, afaics, should do it. Alternatively we can just
publish what backend we're waiting for in PGPROC and make deadlock also
look at that; but, while slightly cleaner, that looks like being more
invasive.
+Copying locks to workers is also important for the avoidance of undetected +deadlock involving both the parallel processe and other processes.
"processe"
For +example, suppose processes A1 and A2 are cooperating parallel processes and +B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to +finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting +for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there +is no cycle. With lock copying, A2 will also hold the lock on L1 and the +deadlock detector can find the cycle A2 -> B -> A2. As in the case of +deadlocks within the parallel group, undetected deadlock occur if either A1 +or A2 acquired a lock after the start of parallelism and attempted to +retain it beyond the end of parallelism. The prohibitions discussed above +protect us against this case.
I think we'd need to add more restrictions to actually make this
guarantee anything. At the very least it would not only have to be
prohibited to end with a lock held, but also to wait for a worker (or
the leader) backend with a not initially granted lock.
Am I missing something or does the copying currently break deadlock.c?
Because afaics that'll compute lock conflicts in FindLockCycleRecurse()
without being aware of the conflicting lock being granted to two
backends. Won't this at least trigger spurious deadlocks? It might
happen to be without consequence for some reason, but this would, at the
very least, need some very careful review.
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 2015-03-18 12:02:07 -0400, Robert Haas wrote:
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c index cb6f8a3..173f0ba 100644 --- a/src/backend/access/heap/heapam.c +++ b/src/backend/access/heap/heapam.c @@ -2234,6 +2234,17 @@ static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid, CommandId cid, int options) { + /* + * For now, parallel operations are required to be strictly read-only. + * Unlike heap_update() and heap_delete(), an insert should never create + * a combo CID, so it might be possible to relax this restrction, but + * not without more thought and testing. + */ + if (IsInParallelMode()) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TRANSACTION_STATE), + errmsg("cannot insert tuples during a parallel operation"))); +
Minor nitpick: Should we perhaps move the ereport to a separate
function? Akin to PreventTransactionChain()? Seems a bit nicer to not
have the same message everywhere.
+void +DestroyParallelContext(ParallelContext *pcxt) +{ + int i; + + /* + * Be careful about order of operations here! We remove the parallel + * context from the list before we do anything else; otherwise, if an + * error occurs during a subsequent step, we might try to nuke it again + * from AtEOXact_Parallel or AtEOSubXact_Parallel. + */ + dlist_delete(&pcxt->node);
Hm. I'm wondering about this. What if we actually fail below? Like in
dsm_detach() or it's callbacks? Then we'll enter abort again at some
point (during proc_exit at the latest) but will not wait for the child
workers. Right?
+/* + * End-of-subtransaction cleanup for parallel contexts. + * + * Currently, it's forbidden to enter or leave a subtransaction while + * parallel mode is in effect, so we could just blow away everything. But + * we may want to relax that restriction in the future, so this code + * contemplates that there may be multiple subtransaction IDs in pcxt_list. + */ +void +AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId) +{ + while (!dlist_is_empty(&pcxt_list)) + { + ParallelContext *pcxt; + + pcxt = dlist_head_element(ParallelContext, node, &pcxt_list); + if (pcxt->subid != mySubId) + break; + if (isCommit) + elog(WARNING, "leaked parallel context"); + DestroyParallelContext(pcxt); + } +}
+/* + * End-of-transaction cleanup for parallel contexts. + */ +void +AtEOXact_Parallel(bool isCommit) +{ + while (!dlist_is_empty(&pcxt_list)) + { + ParallelContext *pcxt; + + pcxt = dlist_head_element(ParallelContext, node, &pcxt_list); + if (isCommit) + elog(WARNING, "leaked parallel context"); + DestroyParallelContext(pcxt); + } +}
Is there any reason to treat the isCommit case as a warning? To me that
sounds like a pretty much guaranteed programming error. If your're going
to argue that a couple resource leakage warnings do similarly: I don't
think that counts for that much. For one e.g. a leaked tupdesc has much
less consequences, for another IIRC those warnings were introduced
after the fact.
+ * When running as a parallel worker, we place only a single + * TransactionStateData is placed on the parallel worker's + * state stack,
'we place .. is placed'
/* + * EnterParallelMode + */ +void +EnterParallelMode(void) +{ + TransactionState s = CurrentTransactionState; + + Assert(s->parallelModeLevel >= 0); + + ++s->parallelModeLevel; +} + +/* + * ExitParallelMode + */ +void +ExitParallelMode(void) +{ + TransactionState s = CurrentTransactionState; + + Assert(s->parallelModeLevel > 0); + Assert(s->parallelModeLevel > 1 || !ParallelContextActive()); + + --s->parallelModeLevel; + + if (s->parallelModeLevel == 0) + CheckForRetainedParallelLocks(); +}
Hm. Is it actually ok for nested parallel mode to retain locks on their
own? Won't that pose a problem? Or did you do it that way just because
we don't have more state? If so that deserves a comment explaining that
htat's the case and why that's acceptable.
@@ -1769,6 +1904,9 @@ CommitTransaction(void) { TransactionState s = CurrentTransactionState; TransactionId latestXid; + bool parallel; + + parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ /* If we might have parallel workers, clean them up now. */ + if (IsInParallelMode()) + { + CheckForRetainedParallelLocks(); + AtEOXact_Parallel(true); + s->parallelModeLevel = 0; + }
'parallel' looks strange when we're also, rightly so, do stuff like
checking IsInParallelMode(). How about naming it is_parallel_worker or
something?
Sorry, ran out of concentration here. It's been a long day.
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, Mar 18, 2015 at 5:36 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!
Sorry, we don't support parallelism yet. :-)
On 2015-03-18 12:02:07 -0400, Robert Haas wrote:
+Instead, we take a more pragmatic approach: we try to make as many of the +operations that are safe outside of parallel mode work correctly in parallel +mode as well, and we try to prohibit the rest via suitable error checks.I'd say that we'd try to prohibit a bunch of common cases. Among them
the ones that are triggerable from SQL. We don't really try to prohibit
many kinds of traps, as you describe.
I revised the text along these lines.
+ - The values of all GUCs. Accordingly, permanent changes to the value of + any GUC are forbidden while in parallel mode; but temporary changes, + such as entering a function with non-NULL proconfig, are potentially OK."potentially OK" sounds odd to me. Which danger are you seing that isn't
relevan tfor norm
Removed "potentially".
+ - The combo CID mappings. This is needed to ensure consistent answers to + tuple visibility checks. The need to synchronize this data structure is + a major reason why we can't support writes in parallel mode: such writes + might create new combo CIDs, and we have now way to let other workers + (or the initiating backend) know about them.Absolutely *not* in the initial version, but we really need to find a
better solution for this.
I have some ideas, but that's not for right now.
+ - The currently active user ID and security context. Note that this is + the fourth user ID we restore: the initial step of binding to the correct + database also involves restoring the authenticated user ID. When GUC + values are restored, this incidentally sets SessionUserId and OuterUserId + to the correct values. This final step restores CurrentUserId.Ah. That's the answer for above. Could you just move it next to the
other user bit?
Well, I think it's good to keep this in the same order it happens.
That's almost true, with the exception of the libraries, which were
out of order. (I've fixed that now.)
+We could copy the entire transaction state stack, +but most of it would be useless: for example, you can't roll back to a +savepoint from within a parallel worker, and there are no resources to +associated with the memory contexts or resource owners of intermediate +subtransactions.I do wonder if we're not going to have to change this in the not too far
away future. But then, this isn't externally visible at all, so
whatever.
I definitely thought about copying the whole stack, but Heikki
suggested this approach, and I didn't see a reason to argue with it.
As you say, we can change it in the future if it proves problematic,
but so far I don't see a problem.
+At the end of a parallel operation, which can happen either because it +completed successfully or because it was interrupted by an error, parallel +workers associated with that operation exit. In the error case, transaction +abort processing in the parallel leader kills of any remaining workers, and +the parallel leader then waits for them to die.Very slightly awkward because first you talk about successful *or* error
and then about abort processing.
I don't understand what's awkward about that. I make a general
statement about what happens at the end of a parallel operation, and
then the next few sentences follow up by explaining what happens in
the error case, and what happens in the success case.
+ - Cleanup of pg_temp namespaces is not done. The initiating backend is + responsible for this, too.How could a worker have its own pg_temp namespace?
It can't. My point here is that it won't clean up the master's
pg_temp namespace. I'll add an explicit prohibition against accessing
temporary relations and clarify these remarks.
+Lock Management +=============== + +Certain heavyweight locks that the initiating backend holds at the beginning +of parallelism are copied to each worker, which unconditionally acquires them. +The parallel backends acquire - without waiting - each such lock that the +leader holds, even if that lock is self-exclusive. This creates the unusual +situation that a lock which could normally only be held by a single backend +can be shared among several backends in a parallel group. + +Obviously, this presents significant hazards that would not be present in +normal execution. If, for example, a backend were to initiate parallelism +while ReindexIsProcessingIndex() were true for some index, the parallel +backends launched at that time would neither share this state nor be excluded +from accessing the index via the heavyweight lock mechanism. It is therefore +imperative that backends only initiate parallelism from places where it will +be safe for parallel workers to access the relations on which they hold locks. +It is also important that they not afterwards do anything which causes access +to those relations to become unsafe, or at least not until after parallelism +has concluded. The fact that parallelism is strictly read-only means that the +opportunities for such mishaps are few and far between; furthermore, most +operations which present these hazards are DDL operations, which will be +rejected by CheckTableNotInUse() if parallel mode is active. + +Only relation locks, locks on database or shared objects, and perhaps the lock +on our transaction ID are copied to workers."perhaps"?
I just meant "if we have one". Changed to "and any lock on our transaction ID".
Advisory locks are not copied, +but the leader may hold them at the start of parallelism; they cannot +subsequently be manipulated while parallel mode is active. It is not safe to +attempt parallelism while holding a lock of any other type, such as a page, +tuple, or relation extension lock, and such attempts will fail. Although +there is currently no reason to do so, such locks could be taken and released +during parallel mode; they merely cannot be held at the start of parallel +mode, since we would then fail to provide necessary mutual exclusion.Is it really true that no such locks are acquired? What about e.g. hash
indexes? They seem to be acquiring page locks while searching.
Ah, right. I have removed "although ... do so". I was thinking that
until we allow writes it wouldn't come up, but that's wrong.
+Copying locks to workers is important for the avoidance of undetected +deadlock between the initiating process and its parallel workers. If the +initiating process holds a lock on an object at the start of parallelism, +and the worker subsequently attempts to acquire a lock on that same object +and blocks, this will result in an undetected deadlock, because the +initiating process cannot finish the transaction (thus releasing the lock) +until the worker terminates, and the worker cannot acquire the lock while +the initiating process holds it. Locks which the processes involved acquire +and then release during parallelism do not present this hazard; they simply +force the processes involved to take turns accessing the protected resource.I don't think this is a strong guarantee. There very well can be lack of
forward progress if they're waiting on each other in some way. Say the
master backend holds the lock and waits on output from a worker. The
worker then will endlessly wait for the lock to become free. A
deadlock. Or, as another scenario, consider cooperating backends that
both try to send tuples to each other but the queue is full. A deadlock.
The idea is that both the master and the workers are restricted to
locks which they take and release again. The idea is, specifically,
to allow syscache lookups. Taking a lock and then waiting for the
worker is no good, which is why WaitForParallelWorkersToFinish() calls
CheckForRetainedParallelLocks(). That would catch your first
scenario.
Your second scenario seems to me to be a different kind of problem.
If that comes up, what you're going to want to do is rewrite the
workers to avoid the deadlock by using non-blocking messaging. (The
recent discussions of fixing similar deadlocks where a libpq client
and a postgres server are both blocked on write while both output
buffers are full centers around similar issues.) Detecting the
deadlock and aborting is better than nothing, but not by much. In any
case, I don't think it's really this patch's job to prevent deadlocks
in code that doesn't exist today and in no way involves the lock
manager.
To me it seems the deadlock detector has to be enhanced to be able to
see 'waiting for' edges. Independent on how we resolve our difference of
opinion on the copying of locks.It seems to me that this isn't all that hard: Whenever we block waiting
for another backend we enter ourselves on the wait queue to that
backend's virtual transaction. When finished we take the blocking
backend off. That, afaics, should do it. Alternatively we can just
publish what backend we're waiting for in PGPROC and make deadlock also
look at that; but, while slightly cleaner, that looks like being more
invasive.
That's an interesting idea. It would be more flexible than what I've
got here right now, in that parallel backends could take and retain
locks on arbitrary objects, and we'd only error out if it actually
created a deadlock, instead of erroring out because of the potential
for a deadlock under some unlikely circumstances. But it can't be
done with existing lock manager APIs - right now there is no way to
put yourself on a wait queue for a virtual transaction except to try
to acquire a conflicting lock, and that's no good because then you
aren't actually trying to read data from it. You'd need some kind of
API that says "pretend I'm waiting for this lock, but don't really
wait for it", and you'd need to be darn sure that you removed yourself
from the wait queue again before doing any other heavyweight lock
manipulation. Do you have specific thoughts on how to implement this?
+Copying locks to workers is also important for the avoidance of undetected +deadlock involving both the parallel processe and other processes."processe"
Fixed.
For +example, suppose processes A1 and A2 are cooperating parallel processes and +B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to +finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting +for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there +is no cycle. With lock copying, A2 will also hold the lock on L1 and the +deadlock detector can find the cycle A2 -> B -> A2. As in the case of +deadlocks within the parallel group, undetected deadlock occur if either A1 +or A2 acquired a lock after the start of parallelism and attempted to +retain it beyond the end of parallelism. The prohibitions discussed above +protect us against this case.I think we'd need to add more restrictions to actually make this
guarantee anything. At the very least it would not only have to be
prohibited to end with a lock held, but also to wait for a worker (or
the leader) backend with a not initially granted lock.
I don't think so. The latter is safe. The other backend will finish
with the lock and then you get it afterwards.
Am I missing something or does the copying currently break deadlock.c?
Because afaics that'll compute lock conflicts in FindLockCycleRecurse()
without being aware of the conflicting lock being granted to two
backends. Won't this at least trigger spurious deadlocks? It might
happen to be without consequence for some reason, but this would, at the
very least, need some very careful review.
There may be a problem here, but I'm not seeing it. Initially, we're
blocking on a lock that we don't already hold, so it's not one of the
copied ones. If we follow one or more waits-for edges and arrive back
at a copied lock, that's a real deadlock and should be reported as
such.
Here is yet another version of this patch. In addition to the fixes
mentioned above, this version includes some minor rebasing around
recent commits, and also better handling of the case where we discover
that we cannot launch workers after all. This can happen because (1)
dynamic_shared_memory_type=none, (2) the maximum number of DSM
segments supported by the system configuration are already in use, or
(3) the user creates a parallel context with nworkers=0. In any of
those cases, the system will now create a backend-private memory
segment instead of a dynamic shared memory segment, and will skip
steps that don't need to be done in that case. This is obviously an
undesirable scenario. If we choose a parallel sequential scan, we
want it to launch workers and really run in parallel. Hopefully, in
case (1) or case (3), we will avoid choosing a parallel plan in the
first place, but case (2) is pretty hard to avoid completely, as we
have no idea what other processes may or may not be doing with dynamic
shared memory segments ... and, in any case, degrading to non-parallel
execution beats failing outright.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v9.patchbinary/octet-stream; name=parallel-mode-v9.patchDownload
From 7e01c13ac9cb48fe70fadabaa6ce571a472d61aa Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/4] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 55 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 287 ++++++++
src/backend/access/transam/parallel.c | 1025 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 491 ++++++++++++-
src/backend/access/transam/xlog.c | 8 +
src/backend/catalog/namespace.c | 11 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 ++++++-
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 ++
src/backend/utils/time/snapmgr.c | 210 +++++-
src/include/access/parallel.h | 68 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/lock.h | 10 +-
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
40 files changed, 2828 insertions(+), 54 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cb6f8a3..df5d19b 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -42,6 +42,7 @@
#include "access/heapam_xlog.h"
#include "access/hio.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/relscan.h"
#include "access/sysattr.h"
#include "access/transam.h"
@@ -1051,7 +1052,13 @@ relation_open(Oid relationId, LOCKMODE lockmode)
/* Make note that we've accessed a temporary relation */
if (RelationUsesLocalBuffers(r))
+ {
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot access temporary tables during a parallel operation")));
MyXactAccessedTempRel = true;
+ }
pgstat_initstats(r);
@@ -1097,7 +1104,13 @@ try_relation_open(Oid relationId, LOCKMODE lockmode)
/* Make note that we've accessed a temporary relation */
if (RelationUsesLocalBuffers(r))
+ {
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot access temporary tables during a parallel operation")));
MyXactAccessedTempRel = true;
+ }
pgstat_initstats(r);
@@ -2234,6 +2247,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restrction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2665,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3079,6 +3113,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5382,6 +5426,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restrction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..c066fff
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,287 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated parallelism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach. First, we try to make as many of
+the operations that are safe outside of parallel mode work correctly in
+parallel mode as well. Second, we try to prohibit common unsafe operations
+via suitable error checks. These checks are intended to catch 100% of
+unsafe things that a user might do from the SQL interface, but code writen
+in C can do unsafe things that won't trigger these checks. The error checks
+are engaged via EnterParallelMode(), which should be called before creating
+a parallel context, and disarmed via ExitParallelMode(), which should be
+called after all parallel contexts have been destroyed. The most
+significant restriction imposed by parallel mode is that all operations must
+be strictly read-only; we allow no writes to the database and no DDL. We
+might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have now way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+ - Locks on relations and database objects, and any lock held on the
+ transaction's XID. See lock management, below.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the parallel
+leader, each parallel worker ends up with a stack of depth 1. This stack
+entry is marked with the special transaction block state
+TBLOCK_PARALLEL_INPROGRESS so that it's not confused with an ordinary
+toplevel transaction. The XID of this TransactionState is set to the XID of
+the innermost currently-active subtransaction in the initiating backend. The
+initiating backend's toplevel XID, and the XIDs of all current (in-progress
+or subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(), and
+TransactionIdIsCurrentTransactionId() return the same values that they would
+in the initiating backend. We could copy the entire transaction state stack,
+but most of it would be useless: for example, you can't roll back to a
+savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initiating
+backend to exit any transaction or subtransaction that was in progress when
+parallelism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context than was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+At the end of a parallel operation, which can happen either because it
+completed successfully or because it was interrupted by an error, parallel
+workers associated with that operation exit. In the error case, transaction
+abort processing in the parallel leader kills of any remaining workers, and
+the parallel leader then waits for them to die. In the case of a successful
+parallel operation, the parallel leader does not send any signals, but must
+wait for workers to complete and exit of their own volition. In either
+case, it is very important that all workers actually exit before the
+parallel leader cleans up the (sub)transaction in which they were created;
+otherwise, chaos can ensue. For example, if the leader is rolling back the
+transaction that created the relation being scanned by a worker, the
+relation could disappear while the worker is still busy scanning it. That's
+not safe.
+
+Generally, the cleanup performed by each worker at this point is similar to
+top-level commit or abort. Each backend has its own resource owners: buffer
+pins, catcache or relcache reference counts, tuple descriptors, and so on
+are managed separately by each backend, and must free them before exiting.
+There are, however, some important differences between parallel worker
+commit or abort and a real top-level transaction commit or abort. Most
+imporantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. Parallel workers cannot
+ safely access the initiating backend's pg_temp namespace, and should
+ not create one of their own.
+
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+leader holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and any lock on
+our transaction ID are copied to workers. Advisory locks are not copied,
+but the leader may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe
+to attempt parallelism while holding a lock of any other type, such as a
+page, tuple, or relation extension lock, and such attempts will fail. Such
+locks can be taken and released during parallel mode; they merely cannot be
+held at the start of parallel mode, since we would then fail to provide
+necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the parallel processes and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..d2cec8e
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,1025 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/xlog.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
+
+/* 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;
+
+ /* Mutex protects remaining fields. */
+ slock_t mutex;
+
+ /* Track whether workers have attached. */
+ int workers_expected;
+ int workers_attached;
+
+ /* Maximum XactLastRecEnd of any worker. */
+ XLogRecPtr last_xlog_end;
+} 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Pointer to our fixed parallel state. */
+static FixedParallelState *MyFixedParallelState;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be non-negative. */
+ Assert(nworkers >= 0);
+
+ /*
+ * If dynamic shared memory is not available, we won't be able to use
+ * background workers.
+ */
+ if (dynamic_shared_memory_type == DSM_IMPL_NONE)
+ nworkers = 0;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len = 0;
+ Size guc_len = 0;
+ Size combocidlen = 0;
+ Size tsnaplen = 0;
+ Size asnaplen = 0;
+ Size tstatelen = 0;
+ Size lockstatelen = 0;
+ Size segsize = 0;
+ int i;
+ FixedParallelState *fps;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allow space to store the fixed-size parallel state. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /*
+ * Normally, the user will have requested at least one worker process,
+ * but if by chance they have not, we can skip a bunch of things here.
+ */
+ if (pcxt->nworkers > 0)
+ {
+ /* Estimate space for various kinds of state sharing. */
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
+ /* If you add more chunks here, you probably need to add keys. */
+ shm_toc_estimate_keys(&pcxt->estimator, 7);
+
+ /* Estimate space need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint info. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ }
+
+ /*
+ * Create DSM and initialize with new table of contents. But if the user
+ * didn't request any workers, then don't bother creating a dynamic shared
+ * memory segment; instead, just use backend-private memory.
+ *
+ * Also, if we can't create a dynamic shared memory segment because the
+ * maximum number of segments have already been created, then fall back
+ * to backend-private memory, and plan not to use any workers. We hope
+ * this won't happen very often, but it's better to abandon the use of
+ * parallelism than to fail outright.
+ */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ if (pcxt->nworkers != 0)
+ pcxt->seg = dsm_create(segsize, DSM_CREATE_NULL_IF_MAXSEGMENTS);
+ if (pcxt->seg != NULL)
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+ else
+ {
+ pcxt->nworkers = 0;
+ pcxt->private = MemoryContextAlloc(TopMemoryContext, segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC, pcxt->private, segsize);
+ }
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ fps->last_xlog_end = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* We can skip the rest of this if we're not budgeting for any workers. */
+ if (pcxt->nworkers > 0)
+ {
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ char *error_queue_space;
+
+ /* Serialize shared libraries we have loaded. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC settings. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshot and active snapshot. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT,
+ tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Serialize heavyweight locks. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data
+ * should be transmitted via separate (possibly larger?) queues.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ char *start;
+ shm_mq *mq;
+
+ start = error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE;
+ mq = shm_mq_create(start, PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* Skip this if we have no workers. */
+ if (pcxt->nworkers == 0)
+ return;
+
+ /* If we do have workers, we'd better have a DSM segment. */
+ Assert(pcxt->seg != NULL);
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ *
+ * Also, we want to update our notion of XactLastRecEnd based on worker
+ * feedback.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ CheckForRetainedParallelLocks();
+
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ if (pcxt->toc != NULL)
+ {
+ FixedParallelState *fps;
+
+ fps = shm_toc_lookup(pcxt->toc, PARALLEL_KEY_FIXED);
+ if (fps->last_xlog_end > XactLastRecEnd)
+ XactLastRecEnd = fps->last_xlog_end;
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ {
+ dsm_detach(pcxt->seg);
+ pcxt->seg = NULL;
+ }
+
+ /*
+ * If this parallel context is actually in backend-private memory rather
+ * than shared memory, free that memory instead.
+ */
+ if (pcxt->private != NULL)
+ {
+ pfree(pcxt->private);
+ pcxt->private = NULL;
+ }
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ if (pcxt->worker != NULL)
+ {
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+ }
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+ SetLatch(MyLatch);
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *lockstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+ MyFixedParallelState = fps;
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
+
+/*
+ * Update shared memory with the ending location of the last WAL record we
+ * wrote, if it's greater than the value already stored there.
+ */
+void
+ParallelWorkerReportLastRecEnd(XLogRecPtr last_xlog_end)
+{
+ FixedParallelState *fps = MyFixedParallelState;
+
+ Assert(fps != NULL);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->last_xlog_end < last_xlog_end)
+ fps->last_xlog_end = last_xlog_end;
+ SpinLockRelease(&fps->mutex);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1495bb4..4969231 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData is placed on the parallel worker's
+ * state stack, and the XID reflected there will be that of the *innermost*
+ * currently-active subtransaction in the backend that initiated paralllelism.
+ * However, GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,51 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+
+ if (s->parallelModeLevel == 0)
+ CheckForRetainedParallelLocks();
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +927,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1640,6 +1773,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1769,6 +1904,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
+
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1802,7 +1940,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1811,6 +1950,14 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ CheckForRetainedParallelLocks();
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1849,10 +1996,28 @@ CommitTransaction(void)
*/
s->state = TRANS_COMMIT;
- /*
- * Here is where we really truly commit.
- */
- latestXid = RecordTransactionCommit();
+ if (!parallel)
+ {
+ /*
+ * We need to mark our XIDs as commited in pg_clog. This is where we
+ * durably commit.
+ */
+ latestXid = RecordTransactionCommit();
+ }
+ else
+ {
+ /*
+ * We must not mark our XID committed; the parallel master is
+ * responsible for that.
+ */
+ latestXid = InvalidTransactionId;
+
+ /*
+ * Make sure the master will know about any WAL we wrote before it
+ * commits.
+ */
+ ParallelWorkerReportLastRecEnd(XactLastRecEnd);
+ }
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1879,7 +2044,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(parallel ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1927,7 +2093,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -1952,6 +2118,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -1975,6 +2144,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2194,7 +2365,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2219,6 +2390,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2237,6 +2411,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool parallel;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2285,6 +2460,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2308,6 +2484,14 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ ForgetParallelLocks();
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2320,9 +2504,23 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ 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);
+ }
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2340,7 +2538,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (parallel)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2361,7 +2562,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, parallel);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2413,6 +2614,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2466,6 +2671,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2501,11 +2707,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2787,6 +2995,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3176,6 +3385,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3353,6 +3563,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3446,6 +3666,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3475,6 +3705,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3495,6 +3737,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3529,6 +3772,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3552,6 +3807,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3629,6 +3885,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3653,6 +3921,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3741,6 +4010,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3763,6 +4046,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3795,6 +4079,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3817,6 +4113,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3828,6 +4132,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3903,6 +4208,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -3994,6 +4300,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4097,6 +4404,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4250,6 +4564,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4390,6 +4711,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4437,6 +4759,139 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 5; /* iso level, deferrable, top & current XID, XID count */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * We need to save and restore XactDeferrable, XactIsoLevel, and the XIDs
+ * associated with this transaction. The first eight bytes of the result
+ * contain XactDeferrable and XactIsoLevel; the next eight bytes contain the
+ * XID of the top-level transaction and the XID of the current transaction
+ * (or, in each case, InvalidTransactionId if none). After that, the next 4
+ * bytes contain a count of how many additional XIDs follow; this is followed
+ * by all of those XIDs one after another. We emit the XIDs in sorted order
+ * for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 5 * sizeof(TransactionId));
+ result[0] = (TransactionId) XactIsoLevel;
+ result[1] = (TransactionId) XactDeferrable;
+ result[2] = XactTopTransactionId;
+ result[3] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 4) * sizeof(TransactionId));
+ result[4] = nParallelCurrentXids;
+ memcpy(&result[5], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[4] = (TransactionId) nxids;
+ memcpy(&result[5], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactIsoLevel = (int) tstate[0];
+ XactDeferrable = (bool) tstate[1];
+ XactTopTransactionId = tstate[2];
+ CurrentTransactionState->transactionId = tstate[3];
+ nParallelCurrentXids = (int) tstate[4];
+ ParallelCurrentXids = &tstate[5];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4506,6 +4961,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index e2d187f..57ba1a3 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -291,6 +291,14 @@ static TimeLineID curFileTLI;
* end+1 of the last record, and is reset when we end a top-level transaction,
* or start a new one; so it can be used to tell if the current transaction has
* created any XLOG records.
+ *
+ * While in parallel mode, this may not be fully up to date. When committing,
+ * a transaction can assume this covers all xlog records written either by the
+ * user backend or by any parallel worker which was present at any point during
+ * the transaction. But when aborting, or when still in parallel mode, other
+ * parallel backends may have written WAL records at later LSNs than the value
+ * stored here. The parallel leader advances its own copy, when necessary,
+ * in WaitForParallelWorkersToFinish.
*/
static XLogRecPtr ProcLastRecPtr = InvalidXLogRecPtr;
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 1af977c..2f6d697 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -20,6 +20,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/dependency.h"
@@ -3646,6 +3647,12 @@ InitTempTableNamespace(void)
(errcode(ERRCODE_READ_ONLY_SQL_TRANSACTION),
errmsg("cannot create temporary tables during recovery")));
+ /* Parallel workers can't create temporary tables, either. */
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_READ_ONLY_SQL_TRANSACTION),
+ errmsg("cannot create temporary tables in parallel mode")));
+
snprintf(namespaceName, sizeof(namespaceName), "pg_temp_%d", MyBackendId);
namespaceId = get_namespace_oid(namespaceName, true);
@@ -3709,7 +3716,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3719,7 +3726,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 6d316d6..80f5553 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -565,6 +565,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -862,6 +869,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index 13ceffa..3cf94ff 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 6c3eff7..ce49c47 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index b3c0502..557d153 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index cf7524f..fe94c8d 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -989,6 +989,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 8eaec0c..68cc6ed 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1683,6 +1683,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..b19bffa 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 33720e8..ea2a432 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2988,7 +2989,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index d9443b1..848c6e8 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -207,6 +208,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -232,6 +234,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -617,6 +637,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -627,6 +648,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
ExecVacuum(stmt, isTopLevel);
}
break;
@@ -703,6 +725,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -734,6 +757,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 7476a26..46bc1f2 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -692,3 +693,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index 26275bd..9c74ed3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5671,6 +5671,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6975,6 +6989,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..8274f84
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,68 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "access/xlogdefs.h"
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ void *private;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
+#define IsParallelWorker() (ParallelWorkerNumber >= 0)
+
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+extern void ParallelWorkerReportLastRecEnd(XLogRecPtr);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index fdf3ea3..60ed01a 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -78,9 +78,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -321,6 +324,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -357,4 +364,8 @@ extern const char *xact_identify(uint8 info);
extern void ParseCommitRecord(uint8 info, xl_xact_commit *xlrec, xl_xact_parsed_commit *parsed);
extern void ParseAbortRecord(uint8 info, xl_xact_abort *xlrec, xl_xact_parsed_abort *parsed);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index cf5f7d0..f3b005f 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index 1477a6f..bd3e90d 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
On Wed, Mar 18, 2015 at 7:10 PM, Andres Freund <andres@2ndquadrant.com> wrote:
On 2015-03-18 12:02:07 -0400, Robert Haas wrote:
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c index cb6f8a3..173f0ba 100644 --- a/src/backend/access/heap/heapam.c +++ b/src/backend/access/heap/heapam.c @@ -2234,6 +2234,17 @@ static HeapTuple heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid, CommandId cid, int options) { + /* + * For now, parallel operations are required to be strictly read-only. + * Unlike heap_update() and heap_delete(), an insert should never create + * a combo CID, so it might be possible to relax this restrction, but + * not without more thought and testing. + */ + if (IsInParallelMode()) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TRANSACTION_STATE), + errmsg("cannot insert tuples during a parallel operation"))); +Minor nitpick: Should we perhaps move the ereport to a separate
function? Akin to PreventTransactionChain()? Seems a bit nicer to not
have the same message everywhere.
Well, it's not actually the same message. They're all a bit
different. Or mostly all of them. And the variable part is not a
command name, as in the PreventTransactionChain() case, so it would
affect translatability if we did that. I don't actually think this is
a big deal.
+void +DestroyParallelContext(ParallelContext *pcxt) +{ + int i; + + /* + * Be careful about order of operations here! We remove the parallel + * context from the list before we do anything else; otherwise, if an + * error occurs during a subsequent step, we might try to nuke it again + * from AtEOXact_Parallel or AtEOSubXact_Parallel. + */ + dlist_delete(&pcxt->node);Hm. I'm wondering about this. What if we actually fail below? Like in
dsm_detach() or it's callbacks? Then we'll enter abort again at some
point (during proc_exit at the latest) but will not wait for the child
workers. Right?
Right. It's actually pretty hard to recover when things that get
called in the abort path fail, which is why dsm_detach() and
shm_mq_detach_callback() try pretty hard to only do things that are
no-fail. For example, failing to detach a dsm gives a WARNING, not
an ERROR. Now, I did make some attempt in previous patches to ensure
that proc_exit() has some ability to recover even if a callback fails
(cf 001a573a2011d605f2a6e10aee9996de8581d099) but I'm not too sure how
useful that's going to be in practice. I'm open to ideas on how to
improve this.
+void +AtEOXact_Parallel(bool isCommit) +{ + while (!dlist_is_empty(&pcxt_list)) + { + ParallelContext *pcxt; + + pcxt = dlist_head_element(ParallelContext, node, &pcxt_list); + if (isCommit) + elog(WARNING, "leaked parallel context"); + DestroyParallelContext(pcxt); + } +}Is there any reason to treat the isCommit case as a warning? To me that
sounds like a pretty much guaranteed programming error. If your're going
to argue that a couple resource leakage warnings do similarly: I don't
think that counts for that much. For one e.g. a leaked tupdesc has much
less consequences, for another IIRC those warnings were introduced
after the fact.
Yeah, I'm going to argue that. A leaked parallel context is pretty
harmless if there are no workers associated with it. If there are,
it's less harmless, of course, but it doesn't seem to me that making
that an ERROR buys us much. I mean, the transaction is going to end
either way, and a message is going to get printed either way, and it's
a bug either way, so whatever.
+ * When running as a parallel worker, we place only a single + * TransactionStateData is placed on the parallel worker's + * state stack,'we place .. is placed'
Will fix in next update.
+ if (s->parallelModeLevel == 0) + CheckForRetainedParallelLocks(); +}Hm. Is it actually ok for nested parallel mode to retain locks on their
own? Won't that pose a problem? Or did you do it that way just because
we don't have more state? If so that deserves a comment explaining that
htat's the case and why that's acceptable.
The only time it's really a problem to retain locks is if you are
doing WaitForParallelWorkersToFinish(). This is pretty much just a
belt-and-suspenders check to make it easier to notice that you've
goofed. But, sure, removing the if part makes sense. I'll do that in
the next update.
@@ -1769,6 +1904,9 @@ CommitTransaction(void) { TransactionState s = CurrentTransactionState; TransactionId latestXid; + bool parallel; + + parallel = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);+ /* If we might have parallel workers, clean them up now. */ + if (IsInParallelMode()) + { + CheckForRetainedParallelLocks(); + AtEOXact_Parallel(true); + s->parallelModeLevel = 0; + }'parallel' looks strange when we're also, rightly so, do stuff like
checking IsInParallelMode(). How about naming it is_parallel_worker or
something?
Yeah, makes sense. Will do that, too.
Sorry, ran out of concentration here. It's been a long day.
Thanks for the review so far.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On 2015-03-19 14:13:59 -0400, Robert Haas wrote:
On Wed, Mar 18, 2015 at 5:36 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!Sorry, we don't support parallelism yet. :-)
And not even proper sequential work apparently...
+ - The currently active user ID and security context. Note that this is + the fourth user ID we restore: the initial step of binding to the correct + database also involves restoring the authenticated user ID. When GUC + values are restored, this incidentally sets SessionUserId and OuterUserId + to the correct values. This final step restores CurrentUserId.Ah. That's the answer for above. Could you just move it next to the
other user bit?Well, I think it's good to keep this in the same order it happens.
That's almost true, with the exception of the libraries, which were
out of order. (I've fixed that now.)
I don't find this convincing in the least. This should explain a
developer which state he can rely on being shared. For that a sane order
is helpful. In which precise order this happens doesn't seem to matter
for that at all; it's also likely ot get out of date.
+At the end of a parallel operation, which can happen either because it +completed successfully or because it was interrupted by an error, parallel +workers associated with that operation exit. In the error case, transaction +abort processing in the parallel leader kills of any remaining workers, and +the parallel leader then waits for them to die.Very slightly awkward because first you talk about successful *or* error
and then about abort processing.I don't understand what's awkward about that. I make a general
statement about what happens at the end of a parallel operation, and
then the next few sentences follow up by explaining what happens in
the error case, and what happens in the success case.
I seem to have completely overread the "In the error case, " part of the
sentence. Forget what I wrote.
+Copying locks to workers is important for the avoidance of undetected +deadlock between the initiating process and its parallel workers. If the +initiating process holds a lock on an object at the start of parallelism, +and the worker subsequently attempts to acquire a lock on that same object +and blocks, this will result in an undetected deadlock, because the +initiating process cannot finish the transaction (thus releasing the lock) +until the worker terminates, and the worker cannot acquire the lock while +the initiating process holds it. Locks which the processes involved acquire +and then release during parallelism do not present this hazard; they simply +force the processes involved to take turns accessing the protected resource.I don't think this is a strong guarantee. There very well can be lack of
forward progress if they're waiting on each other in some way. Say the
master backend holds the lock and waits on output from a worker. The
worker then will endlessly wait for the lock to become free. A
deadlock. Or, as another scenario, consider cooperating backends that
both try to send tuples to each other but the queue is full. A deadlock.The idea is that both the master and the workers are restricted to
locks which they take and release again. The idea is, specifically,
to allow syscache lookups. Taking a lock and then waiting for the
worker is no good, which is why WaitForParallelWorkersToFinish() calls
CheckForRetainedParallelLocks(). That would catch your first
scenario.Your second scenario seems to me to be a different kind of problem.
If that comes up, what you're going to want to do is rewrite the
workers to avoid the deadlock by using non-blocking messaging.
I don't think that actually really solves the problem. You'll still end
up blocking with full "pipes" at some point. Whether you do it inside
the messaging or outside the messaging code doesn't particularly matter.
In any case, I don't think it's really this patch's job to prevent
deadlocks in code that doesn't exist today and in no way involves the
lock manager.
Well, you're arguing that we need a solution in the parallelism
infrastructure for the lock deadlocks problem. I don't think it's absurd
to extend care to other cases.
To me it seems the deadlock detector has to be enhanced to be able to
see 'waiting for' edges. Independent on how we resolve our difference of
opinion on the copying of locks.It seems to me that this isn't all that hard: Whenever we block waiting
for another backend we enter ourselves on the wait queue to that
backend's virtual transaction. When finished we take the blocking
backend off. That, afaics, should do it. Alternatively we can just
publish what backend we're waiting for in PGPROC and make deadlock also
look at that; but, while slightly cleaner, that looks like being more
invasive.That's an interesting idea. It would be more flexible than what I've
got here right now, in that parallel backends could take and retain
locks on arbitrary objects, and we'd only error out if it actually
created a deadlock, instead of erroring out because of the potential
for a deadlock under some unlikely circumstances.
It also seems like it'd be able to deal with a bunch of scenarios that
the current approach wouldn't be able to deal with.
But it can't be
done with existing lock manager APIs - right now there is no way to
put yourself on a wait queue for a virtual transaction except to try
to acquire a conflicting lock, and that's no good because then you
aren't actually trying to read data from it.
Right.
You'd need some kind of
API that says "pretend I'm waiting for this lock, but don't really
wait for it", and you'd need to be darn sure that you removed yourself
from the wait queue again before doing any other heavyweight lock
manipulation. Do you have specific thoughts on how to implement this?
I've thought some about this, and I think it's a bit easier to not do it
on the actual lock waitqueues, but teach deadlock.c about that kind of
blocking.
deadlock.c is far from simple, and at least I don't find the control
flow to be particularly clear. So it's not easy. It'd be advantageous
to tackle things at that level because it'd avoid the need to acquire
locks on some lock's waitqueue when blocking; we're going to do that a
lot.
But It seems to me that it should be possible to suceed: In
FindLockCycleRecurse(), in the case that we're not waiting for an actual
lock (checkProc->links.next == NULL) we can add a case that considers
the 'blocking parallelism' case. ISTM that that's just a
FindLockCycleRecurse() call on the process that we're waiting for. We'd
either have to find the other side's locktag for DEADLOCK_INFO or invent
another field in there; but that seems like a solveable problem.
Am I missing something or does the copying currently break deadlock.c?
Because afaics that'll compute lock conflicts in FindLockCycleRecurse()
without being aware of the conflicting lock being granted to two
backends. Won't this at least trigger spurious deadlocks? It might
happen to be without consequence for some reason, but this would, at the
very least, need some very careful review.There may be a problem here, but I'm not seeing it. Initially, we're
blocking on a lock that we don't already hold, so it's not one of the
copied ones. If we follow one or more waits-for edges and arrive back
at a copied lock, that's a real deadlock and should be reported as
such.
That's a fair point. I was worried that this is going to introduce
additional hard edges between processes. I think it actually might, but
only when a lock is upgraded; which really shouldn't happen for the
copied locks.
Also: Man, trying to understand the guts of deadlock.c only made me
understand how *friggin* expensive deadlock checking is. I'm really
rather surprised that it only infrequently causes problems. I think I
have seen a report or two where it might have been the deadlock check
that went bonkers during schema upgrades on larger setups, but that's
it.
I'm not sure if I said that somewhere before: If we aborted parallelism
if any of the to-be-copied locks would conflict with its copy, instead
of duplicating them, I could live with this. Then this would amount to
jumping the lock queue, which seems reasonable to me. Since not being
able to do parallelism already needs to be handled gracefull, this
doesn't seem to be too bad.
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
Andres Freund <andres@2ndquadrant.com> writes:
Also: Man, trying to understand the guts of deadlock.c only made me
understand how *friggin* expensive deadlock checking is. I'm really
rather surprised that it only infrequently causes problems.
The reason for that is that we only run deadlock checking if something's
been waiting for at least one second, which pretty much takes it out
of any performance-relevant code pathway. I think it would be a serious
error to put any deadlock-checking-like behavior into mainline code.
Which probably means that Andres is right that teaching deadlock.c
about any new sources of deadlock is the way to approach this.
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, Mar 24, 2015 at 3:26 PM, Andres Freund <andres@2ndquadrant.com> wrote:
+ - The currently active user ID and security context. Note that this is + the fourth user ID we restore: the initial step of binding to the correct + database also involves restoring the authenticated user ID. When GUC + values are restored, this incidentally sets SessionUserId and OuterUserId + to the correct values. This final step restores CurrentUserId.Ah. That's the answer for above. Could you just move it next to the
other user bit?Well, I think it's good to keep this in the same order it happens.
That's almost true, with the exception of the libraries, which were
out of order. (I've fixed that now.)I don't find this convincing in the least. This should explain a
developer which state he can rely on being shared. For that a sane order
is helpful. In which precise order this happens doesn't seem to matter
for that at all; it's also likely ot get out of date.
I'm hoping we can agree to disagree on this point. I like my order
order better, you like your order better; we're arguing about the
ordering of paragraphs in a README.
You'd need some kind of
API that says "pretend I'm waiting for this lock, but don't really
wait for it", and you'd need to be darn sure that you removed yourself
from the wait queue again before doing any other heavyweight lock
manipulation. Do you have specific thoughts on how to implement this?I've thought some about this, and I think it's a bit easier to not do it
on the actual lock waitqueues, but teach deadlock.c about that kind of
blocking.deadlock.c is far from simple, and at least I don't find the control
flow to be particularly clear. So it's not easy. It'd be advantageous
to tackle things at that level because it'd avoid the need to acquire
locks on some lock's waitqueue when blocking; we're going to do that a
lot.But It seems to me that it should be possible to suceed: In
FindLockCycleRecurse(), in the case that we're not waiting for an actual
lock (checkProc->links.next == NULL) we can add a case that considers
the 'blocking parallelism' case. ISTM that that's just a
FindLockCycleRecurse() call on the process that we're waiting for. We'd
either have to find the other side's locktag for DEADLOCK_INFO or invent
another field in there; but that seems like a solveable problem.
I'll take a look at this. Thanks for the pointers.
That's a fair point. I was worried that this is going to introduce
additional hard edges between processes. I think it actually might, but
only when a lock is upgraded; which really shouldn't happen for the
copied locks.
Agreed. And if it does, and we get a deadlock, I think I'm sorta OK
with that. It's a known fact that lock upgrades are a deadlock
hazard, and the possible existence of obscure scenarios where that's
more likely when parallelism is involved than without it doesn't
bother me terribly. I mean, we have to look at the specifics, but I
think it's far from obvious that
I'm not sure if I said that somewhere before: If we aborted parallelism
if any of the to-be-copied locks would conflict with its copy, instead
of duplicating them, I could live with this. Then this would amount to
jumping the lock queue, which seems reasonable to me. Since not being
able to do parallelism already needs to be handled gracefull, this
doesn't seem to be too bad.
It's a tempting offer since it would shorten the path to getting this
committed, but I think that's leaving a lot on the table. In
particular, if we ever want to have parallel CLUSTER, VACUUM FULL, or
ALTER TABLE, that's not going to get us there. If we don't have the
right infrastructure to support that in whatever the initial commit
is, fine. But we've got to have a plan that gets us there, or we're
just boxing ourselves into a corner.
--
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 wrote:
On Wed, Mar 18, 2015 at 7:10 PM, Andres Freund <andres@2ndquadrant.com> wrote:
+ /* + * For now, parallel operations are required to be strictly read-only. + * Unlike heap_update() and heap_delete(), an insert should never create + * a combo CID, so it might be possible to relax this restrction, but + * not without more thought and testing. + */ + if (IsInParallelMode()) + ereport(ERROR, + (errcode(ERRCODE_INVALID_TRANSACTION_STATE), + errmsg("cannot insert tuples during a parallel operation"))); +Minor nitpick: Should we perhaps move the ereport to a separate
function? Akin to PreventTransactionChain()? Seems a bit nicer to not
have the same message everywhere.Well, it's not actually the same message. They're all a bit
different. Or mostly all of them. And the variable part is not a
command name, as in the PreventTransactionChain() case, so it would
affect translatability if we did that.
Three things that vary are 1) the fact that some check IsParallelWorker()
and others check IsInParallelMode(), and 2) that some of them are
ereports() while others are elog(), and 3) that some are ERROR and
others are FATAL. Is there a reason for these differences?
(Note typo "restrction" in quoted paragraph above.)
Maybe something similar to what commit f88d4cfc did: have a function
where all possible messages are together, and one is selected with some
enum. That way, it's easier to maintain consistency if more cases are
added in the future.
I don't actually think this is a big deal.
Yeah.
--
�lvaro Herrera http://www.2ndQuadrant.com/
PostgreSQL Development, 24x7 Support, Remote DBA, 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 Thu, Mar 19, 2015 at 11:13 AM, Robert Haas <robertmhaas@gmail.com> wrote:
Here is yet another version of this patch. In addition to the fixes
mentioned above, this version includes some minor rebasing around
recent commits, and also better handling of the case where we discover
that we cannot launch workers after all. This can happen because (1)
dynamic_shared_memory_type=none, (2) the maximum number of DSM
segments supported by the system configuration are already in use, or
(3) the user creates a parallel context with nworkers=0. In any of
those cases, the system will now create a backend-private memory
segment instead of a dynamic shared memory segment, and will skip
steps that don't need to be done in that case. This is obviously an
undesirable scenario. If we choose a parallel sequential scan, we
want it to launch workers and really run in parallel. Hopefully, in
case (1) or case (3), we will avoid choosing a parallel plan in the
first place, but case (2) is pretty hard to avoid completely, as we
have no idea what other processes may or may not be doing with dynamic
shared memory segments ... and, in any case, degrading to non-parallel
execution beats failing outright.
I see that you're using git format-patch to generate this. But the
patch is only patch 1/4. Is that intentional? Where are the other
pieces?
I think that the parallel seqscan patch, and the assessing parallel
safety patch are intended to fit together with this patch, but I can't
find a place where there is a high level overview explaining just how
they fit together (I notice Amit's patch has an "#include
"access/parallel.h", which is here, but that wasn't trivial to figure
out). I haven't been paying too much attention to this patch series.
--
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 Tue, Mar 24, 2015 at 3:26 PM, Andres Freund <andres@2ndquadrant.com> wrote:
You'd need some kind of
API that says "pretend I'm waiting for this lock, but don't really
wait for it", and you'd need to be darn sure that you removed yourself
from the wait queue again before doing any other heavyweight lock
manipulation. Do you have specific thoughts on how to implement this?I've thought some about this, and I think it's a bit easier to not do it
on the actual lock waitqueues, but teach deadlock.c about that kind of
blocking.deadlock.c is far from simple, and at least I don't find the control
flow to be particularly clear. So it's not easy. It'd be advantageous
to tackle things at that level because it'd avoid the need to acquire
locks on some lock's waitqueue when blocking; we're going to do that a
lot.But It seems to me that it should be possible to suceed: In
FindLockCycleRecurse(), in the case that we're not waiting for an actual
lock (checkProc->links.next == NULL) we can add a case that considers
the 'blocking parallelism' case. ISTM that that's just a
FindLockCycleRecurse() call on the process that we're waiting for. We'd
either have to find the other side's locktag for DEADLOCK_INFO or invent
another field in there; but that seems like a solveable problem.
I (finally) had some time to look at this today. Initially, it looked
good. Unfortunately, the longer I looked at it, the less convinced I
got that we could solve the problem this way. The core of the issue
is that the Funnel node in the parallel group leader basically does
this:
while (!local_scan_done || !remote_scan_done)
{
attempt a read from each remaining worker's tuple queue, blocking
only if local_scan_done;
if (we got a tuple)
return it;
else if (there are no remaining workers)
remote_scan_done = true;
attempt to produce a tuple just as if we were a worker ourselves;
if (we got a tuple)
return it;
else
local_scan_done = true;
}
Imagine that we're doing a parallel sequential scan; each worker
claims one page but goes into the tank before it has returned all of
the tuples on that page. The master reads all the remaining pages but
must now wait for the workers to finish returning the tuples on the
pages they claimed.
So what this means is:
1. The master doesn't actually *wait* until the very end of the parallel phase.
2. When the master does wait, it waits for all of the parallel workers
at once, not each one individually.
So, I don't think anything as simplistic as teaching a blocking
shm_mq_receive() to tip off the deadlock detector that we're waiting
for the process on the other end of that particular queue can ever
work. Because of point #2, that never happens. When I first started
thinking about how to fix this, I said, well, that's no big deal, we
can just advertise the whole list of processes that we're waiting for
in shared memory, rather than just the one. This is a bit tricky,
though. Any general API for any backend to advertise that it's waiting
for an arbitrary subset of the other backends would require O(n^2)
shared memory state. That wouldn't be completely insane, but it's not
great, either. For this particular case, we could optimize that down
to O(n) by just chaining all of the children of a given parallel group
leader in a linked whose nodes are inlined in their PGPROCs, but that
doesn't feel very general, because it forecloses the possibility of
the children ever using that API, and I think they might need to. If
nothing else, they might need to advertise that they're blocking on
the master if they are trying to send data, the queue is full, and
they have to wait for the master to drain some of it before they can
proceed.
After thinking about it a bit more, I realized that even if we settle
on some solution to that problem, there's another issues: the
wait-edges created by this system don't really have the same semantics
as regular lock waits. Suppose A is waiting on a lock held by B and B
is waiting for a lock held by A; that's a deadlock. But if A is
waiting for B to write to a tuple queue and B is waiting for A to read
from a tuple queue, that's not a deadlock if the queues in question
are the same. If they are different queues, it might be a deadlock,
but then again maybe not. It may be that A is prepared to accept B's
message from one queue, and that upon fully receiving it, it will do
some further work that will lead it to write a tuple into the other
queue. If so, we're OK; if not, it's a deadlock. I'm not sure
whether you'll want to argue that that is an implausible scenario, but
I'm not too sure it is. The worker could be saying "hey, I need some
additional piece of your backend-local state in order to finish this
computation", and the master could then provide it. I don't have any
plans like that, but it's been suggested previously by others, so it's
not an obviously nonsensical thing to want to do.
A further difference in the semantics of these wait-edges is that if
process A is awaiting AccessExclusiveLock on a resource held by B, C,
and D at AccessShareLock, it needs to wait for all of those processes
to release their locks before it can do anything else. On the other
hand, if process A is awaiting tuples from B, C, and D, it just needs
ONE of those processes to emit tuples in order to make progress. Now
maybe that doesn't make any difference in practice, because even if
two of those processes are making lively progress and A is receiving
tuples from them and processing them like gangbusters, that's probably
not going to help the third one get unstuck. If we adopt the approach
of discounting that possibility, then as long as the parallel leader
can generate tuples locally (that is, local_scan_done = false) we
don't report the deadlock, but as soon as it can no longer do that
(local_scan_done = true) then we do, even though we could still
theoretically read more tuples from the non-stuck workers. So then
you have to wonder why we're not solving problem #1, because the
deadlock was just as certain before we generated the maximum possible
number of tuples locally as it was afterwards.
Thoughts?
--
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 24, 2015 at 11:56 PM, Alvaro Herrera
<alvherre@2ndquadrant.com> wrote:
Well, it's not actually the same message. They're all a bit
different. Or mostly all of them. And the variable part is not a
command name, as in the PreventTransactionChain() case, so it would
affect translatability if we did that.Three things that vary are 1) the fact that some check IsParallelWorker()
and others check IsInParallelMode(), and 2) that some of them are
ereports() while others are elog(), and 3) that some are ERROR and
others are FATAL. Is there a reason for these differences?
The message text also varies, because it states the particular
operation that is prohibited.
We should check IsParallelWorker() for operations that are allowed in
the master during parallel mode, but not allowed in the workers - e.g.
the master can scan its own temporary relations, but its workers
can't. We should check IsInParallelMode() for operations that are
completely off-limits in parallel mode - i.e. writes.
We use ereport() where we expect that SQL could hit that check, and
elog() where we expect that only (buggy?) C code could hit that check.
We use FATAL for some of the checks in xact.c, for parity with other,
similar checks in xact.c that relate to checking the transaction
state. I think this is because we assume that if the transaction
state is hosed, it's necessary to terminate the backend to recover.
In other cases, we use ERROR.
(Note typo "restrction" in quoted paragraph above.)
Thanks, will fix.
--
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, Apr 20, 2015 at 6:49 PM, Peter Geoghegan <pg@heroku.com> wrote:
I see that you're using git format-patch to generate this. But the
patch is only patch 1/4. Is that intentional? Where are the other
pieces?I think that the parallel seqscan patch, and the assessing parallel
safety patch are intended to fit together with this patch, but I can't
find a place where there is a high level overview explaining just how
they fit together (I notice Amit's patch has an "#include
"access/parallel.h", which is here, but that wasn't trivial to figure
out). I haven't been paying too much attention to this patch series.
The intended order of application is:
- parallel mode/contexts
- assess parallel safety
- parallel heap scan
- parallel seq scan
The parallel heap scan patch should probably be merged into the
parallel seq scan patch, which should probably then be split into
several patches, one or more with general parallel query
infrastructure and then another with stuff specific to parallel
sequential scan. But we're not quite there yet. Until we can get
agreement on how to finalize the parallel mode/contexts patch, I
haven't been too worried about getting the other stuff sorted out
exactly 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 Tue, Apr 21, 2015 at 11:38 PM, Robert Haas <robertmhaas@gmail.com> wrote:
After thinking about it a bit more, I realized that even if we settle
on some solution to that problem, there's another issues: the
wait-edges created by this system don't really have the same semantics
as regular lock waits. Suppose A is waiting on a lock held by B and B
is waiting for a lock held by A; that's a deadlock. But if A is
waiting for B to write to a tuple queue and B is waiting for A to read
from a tuple queue, that's not a deadlock if the queues in question
are the same. If they are different queues, it might be a deadlock,
but then again maybe not. It may be that A is prepared to accept B's
message from one queue, and that upon fully receiving it, it will do
some further work that will lead it to write a tuple into the other
queue. If so, we're OK; if not, it's a deadlock.
I agree that the way deadlock detector works for locks, it might not
be same as it needs to work for communication buffers (tuple queues).
Here I think we also need to devise some different way to remove resources
from wait/resource queue, it might not be a good idea to do it similar to
locks
as locks are released at transaction end whereas this new resources
(communication buffers) don't operate at transaction boundary.
I'm not sure
whether you'll want to argue that that is an implausible scenario, but
I'm not too sure it is. The worker could be saying "hey, I need some
additional piece of your backend-local state in order to finish this
computation", and the master could then provide it. I don't have any
plans like that, but it's been suggested previously by others, so it's
not an obviously nonsensical thing to want to do.
If such a thing is not possible today and also it seems that is a not a
good design to solve any problem, then why to spend too much effort
in trying to find ways to detect deadlocks for the same.
A further difference in the semantics of these wait-edges is that if
process A is awaiting AccessExclusiveLock on a resource held by B, C,
and D at AccessShareLock, it needs to wait for all of those processes
to release their locks before it can do anything else. On the other
hand, if process A is awaiting tuples from B, C, and D, it just needs
ONE of those processes to emit tuples in order to make progress. Now
maybe that doesn't make any difference in practice, because even if
two of those processes are making lively progress and A is receiving
tuples from them and processing them like gangbusters, that's probably
not going to help the third one get unstuck. If we adopt the approach
of discounting that possibility, then as long as the parallel leader
can generate tuples locally (that is, local_scan_done = false) we
don't report the deadlock, but as soon as it can no longer do that
(local_scan_done = true) then we do, even though we could still
theoretically read more tuples from the non-stuck workers. So then
you have to wonder why we're not solving problem #1, because the
deadlock was just as certain before we generated the maximum possible
number of tuples locally as it was afterwards.
I think the deadlock detection code should run if anytime we have to
wait for more than deadlock timeout. Now I think the important point
here is when to actually start waiting, as per current parallel_seqscan
patch we wait only after checking the tuples from all queues, we could
have done it other way (wait if we can't fetch from one queue) as well.
It seems both has pros and cons, so we can proceed assuming current
way is okay and we can consider to change it in future once we find some
reason for the same.
Having said above, I feel this is not the problem that we should try to
solve at this point unless there is any scenario where we could hit
deadlock due to communication buffers. I think such a solution would
be required for advanced form of parallelism (like intra-query parallelism).
By the way, why are we trying to solve this problem, is there any way
with which we can hit it for parallel sequential scan?
With Regards,
Amit Kapila.
EnterpriseDB: http://www.enterprisedb.com
On 19-03-2015 15:13, Robert Haas wrote:
On Wed, Mar 18, 2015 at 5:36 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!
I'm also looking at this piece of code and found some low-hanging fruit.
--
Euler Taveira Timbira - http://www.timbira.com.br/
PostgreSQL: Consultoria, Desenvolvimento, Suporte 24x7 e Treinamento
Attachments:
0001-fix-some-typos.patchtext/x-patch; name=0001-fix-some-typos.patchDownload
>From 3ce5376868f61a67540915b83a15c59a31fc895a Mon Sep 17 00:00:00 2001
From: Euler Taveira <euler@timbira.com>
Date: Sun, 26 Apr 2015 13:49:37 -0300
Subject: [PATCH 1/3] fix some typos.
---
src/backend/access/heap/heapam.c | 8 ++++----
1 file changed, 4 insertions(+), 4 deletions(-)
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index da0b70e..16d8c59 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2250,7 +2250,7 @@ heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
/*
* For now, parallel operations are required to be strictly read-only.
* Unlike heap_update() and heap_delete(), an insert should never create
- * a combo CID, so it might be possible to relax this restrction, but
+ * a combo CID, so it might be possible to relax this restriction, but
* not without more thought and testing.
*/
if (IsInParallelMode())
@@ -2666,7 +2666,7 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
/*
- * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Forbid this during a parallel operation, lets it allocate a combocid.
* Other workers might need that combocid for visibility checks, and we
* have no provision for broadcasting it to them.
*/
@@ -3124,7 +3124,7 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
- * Forbid this during a parallel operation, lest it allocate a combocid.
+ * Forbid this during a parallel operation, lets it allocate a combocid.
* Other workers might need that combocid for visibility checks, and we
* have no provision for broadcasting it to them.
*/
@@ -5435,7 +5435,7 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
/*
* For now, parallel operations are required to be strictly read-only.
* Unlike a regular update, this should never create a combo CID, so it
- * might be possible to relax this restrction, but not without more
+ * might be possible to relax this restriction, but not without more
* thought and testing. It's not clear that it would be useful, anyway.
*/
if (IsInParallelMode())
--
2.1.4
0002-Avoid-compiler-warnings-about-unused-variables-in-as.patchtext/x-patch; name=0002-Avoid-compiler-warnings-about-unused-variables-in-as.patchDownload
>From cf25445d9d21496b6927e9ef45e6c3815fef8ad5 Mon Sep 17 00:00:00 2001
From: Euler Taveira <euler@timbira.com>
Date: Sun, 26 Apr 2015 14:24:26 -0300
Subject: [PATCH 2/3] Avoid compiler warnings about unused variables in
assert-disabled builds.
---
src/backend/utils/fmgr/funcapi.c | 2 +-
1 file changed, 1 insertion(+), 1 deletion(-)
diff --git a/src/backend/utils/fmgr/funcapi.c b/src/backend/utils/fmgr/funcapi.c
index ebd7ddd..b9f2b06 100644
--- a/src/backend/utils/fmgr/funcapi.c
+++ b/src/backend/utils/fmgr/funcapi.c
@@ -887,7 +887,7 @@ get_func_trftypes(HeapTuple procTup,
Oid **p_trftypes)
{
- Form_pg_proc procStruct = (Form_pg_proc) GETSTRUCT(procTup);
+ Form_pg_proc procStruct PG_USED_FOR_ASSERTS_ONLY = (Form_pg_proc) GETSTRUCT(procTup);
Datum protrftypes;
ArrayType *arr;
int nelems;
--
2.1.4
0003-Fix-some-more-typos.patchtext/x-patch; name=0003-Fix-some-more-typos.patchDownload
>From 7d1716fdf84f24a1dddfa02db27d532e06c92c3d Mon Sep 17 00:00:00 2001
From: Euler Taveira <euler@timbira.com>
Date: Sun, 26 Apr 2015 14:52:39 -0300
Subject: [PATCH 3/3] Fix some more typos.
---
src/backend/access/transam/README.parallel | 6 +++---
1 file changed, 3 insertions(+), 3 deletions(-)
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
index c066fff..2257e4c 100644
--- a/src/backend/access/transam/README.parallel
+++ b/src/backend/access/transam/README.parallel
@@ -76,7 +76,7 @@ Instead, we take a more pragmatic approach. First, we try to make as many of
the operations that are safe outside of parallel mode work correctly in
parallel mode as well. Second, we try to prohibit common unsafe operations
via suitable error checks. These checks are intended to catch 100% of
-unsafe things that a user might do from the SQL interface, but code writen
+unsafe things that a user might do from the SQL interface, but code written
in C can do unsafe things that won't trigger these checks. The error checks
are engaged via EnterParallelMode(), which should be called before creating
a parallel context, and disarmed via ExitParallelMode(), which should be
@@ -108,7 +108,7 @@ worker. This includes:
- The combo CID mappings. This is needed to ensure consistent answers to
tuple visibility checks. The need to synchronize this data structure is
a major reason why we can't support writes in parallel mode: such writes
- might create new combo CIDs, and we have now way to let other workers
+ might create new combo CIDs, and we have no way to let other workers
(or the initiating backend) know about them.
- The transaction snapshot.
@@ -178,7 +178,7 @@ pins, catcache or relcache reference counts, tuple descriptors, and so on
are managed separately by each backend, and must free them before exiting.
There are, however, some important differences between parallel worker
commit or abort and a real top-level transaction commit or abort. Most
-imporantly:
+importantly:
- No commit or abort record is written; the initiating backend is
responsible for this.
--
2.1.4
On 2015-04-22 AM 04:14, Robert Haas wrote:
We should check IsParallelWorker() for operations that are allowed in
the master during parallel mode, but not allowed in the workers - e.g.
the master can scan its own temporary relations, but its workers
can't. We should check IsInParallelMode() for operations that are
completely off-limits in parallel mode - i.e. writes.We use ereport() where we expect that SQL could hit that check, and
elog() where we expect that only (buggy?) C code could hit that check.
By the way, perhaps orthogonal to the basic issue Alvaro and you are
discussing here - when playing around with (parallel-mode + parallel-safety +
parallel heapscan + parallel seqscan), I'd observed (been a while since) that
if you run a CREATE TABLE AS ... SELECT and the SELECT happens to use parallel
scan, the statement as a whole fails because the top level statement (CTAS) is
not read-only. So, only way to make CTAS succeed is to disable seqscan; which
may require some considerations in reporting to user to figure out. I guess it
happens because the would-be parallel leader of the scan would also be the one
doing DefineRelation() DDL. Apologies if this has been addressed since.
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 Sun, Apr 26, 2015 at 2:03 PM, Euler Taveira <euler@timbira.com.br> wrote:
On 19-03-2015 15:13, Robert Haas wrote:
On Wed, Mar 18, 2015 at 5:36 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!I'm also looking at this piece of code and found some low-hanging fruit.
Thanks. 0001 and 0003 look good, but 0002 is actually unrelated to this patch.
--
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, Apr 26, 2015 at 9:57 PM, Amit Langote
<Langote_Amit_f8@lab.ntt.co.jp> wrote:
On 2015-04-22 AM 04:14, Robert Haas wrote:
We should check IsParallelWorker() for operations that are allowed in
the master during parallel mode, but not allowed in the workers - e.g.
the master can scan its own temporary relations, but its workers
can't. We should check IsInParallelMode() for operations that are
completely off-limits in parallel mode - i.e. writes.We use ereport() where we expect that SQL could hit that check, and
elog() where we expect that only (buggy?) C code could hit that check.By the way, perhaps orthogonal to the basic issue Alvaro and you are
discussing here - when playing around with (parallel-mode + parallel-safety +
parallel heapscan + parallel seqscan), I'd observed (been a while since) that
if you run a CREATE TABLE AS ... SELECT and the SELECT happens to use parallel
scan, the statement as a whole fails because the top level statement (CTAS) is
not read-only. So, only way to make CTAS succeed is to disable seqscan; which
may require some considerations in reporting to user to figure out. I guess it
happens because the would-be parallel leader of the scan would also be the one
doing DefineRelation() DDL. Apologies if this has been addressed since.
That sounds like a bug in the assess-parallel-safety patch.
--
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 2:38 PM, Robert Haas <robertmhaas@gmail.com> wrote:
On Sun, Apr 26, 2015 at 2:03 PM, Euler Taveira <euler@timbira.com.br> wrote:
On 19-03-2015 15:13, Robert Haas wrote:
On Wed, Mar 18, 2015 at 5:36 PM, Andres Freund <andres@2ndquadrant.com> wrote:
Reading the README first, the rest later. So you can comment on my
comments, while I actually look at the code. Parallelism, yay!I'm also looking at this piece of code and found some low-hanging fruit.
Thanks. 0001 and 0003 look good, but 0002 is actually unrelated to this patch.
So, I think it makes sense to split up this patch in two. There's no
real debate, AFAICS, about anything in the patch other than the
heavyweight locking stuff. So I'd like to go ahead and commit the
rest. That's attached here as parallel-mode-v10.patch.
The remaining question here is what to do about the heavyweight
locking. There seem to be a couple of possible approaches to that,
and perhaps with this specific issue separated out we can focus in on
that aspect of the problem. Let me start by restating my goals for
that portion of the patch:
1. Avoid undetected deadlocks. For example, suppose A and A1 are
cooperating parallel processes, and B is an unrelated process. If A1
waits for a lock held by B, and B waits for a lock held by A, and A is
waiting for A1 to finish executing before winding up parallelism, the
deadlock detector currently won't detect that. Either the deadlock
detector needs to know that A waits for A1, or it needs to view the
lock held by A as also held by A1, so that it can detect a cycle A1 ->
B -> A1.
2. Avoid unprincipled deadlocks. For example, suppose A holds
AccessExclusiveLock on a relation R and attempts a parallel sequential
scan on R. It launches a worker A1. If A1 attempts to lock R and
blocks, and A then waits for A1, we have a deadlock. Detecting this
deadlock (as per goal 1) is better than not detecting it, but it's not
good enough. Either A shouldn't have attempted a parallel sequential
scan in the first place, or it should run to completion without
deadlocking with its own workers.
3. Allow parallel DDL. For example, parallel VACUUM FULL or parallel
CLUSTER. We don't have code for these things today, but the locking
architecture we choose should not preclude doing them later.
4. Avoid restricting parallelism based on the history of the
transaction. If BEGIN; COPY foo FROM ...; SELECT ... FROM foo can use
parallelism, then BEGIN; TRUNCATE foo; COPY foo FROM ...; SELECT ...
FROM foo should also be able to use parallelism.
5. Impose as few restriction as possible on what can be done in
parallel mode. For example, the current version of this patch
(attached as parallel-mode-locking.patch) allows parallel workers to
do system cache lookups, where a relation lock is taken and released,
but they cannot take and retain any heavyweight locks; so for example
a user-defined function that runs on SQL query against some unrelated
table and returns the results isn't parallel-safe right now. This is
not ideal.
That's all I've got for goals. How do we achieve those goals? So far
I think we've hit on three approaches to deadlock detection that I
think are at least somewhat plausible. None of them are perfect:
D-1. Teach the deadlock detector about implicit edges between
cooperating parallel processes. Upthread, I wrote about some problems
with this approach:
/messages/by-id/CA+TgmoZ0tOPvfuz5BmEUqmdQ9xYQHzkG4jjJXduYDgQJXmPFbQ@mail.gmail.com
D-2. Copy locks from the master to every worker, and forbid workers
from acquiring any new, long-lived locks. (Locks that are taken and
released quickly, like a system catalog lock for a syscache lookup, or
a relation extension lock, won't cause deadlock.) Copying locks
ensures that whenever there is a dangerous structure like A1 -> B -> A
in the waits-for graph, there will also be a cycle A1 -> B -> A1, so
the unmodified deadlock detector will detect the deadlock. Forbidding
workers from acquiring any new, long-lived locks prevents locks taken
after the copying step from causing similar problems. The restriction
against new, long-lived locks is annoying. Also, a transaction
holding many locks (like pg_dump) and using many workers will use an
enormous number of lock table slots.
D-3. Teach the deadlock detector to consider all locks held by any
member of the locking group as held by the leader. This guarantees
that all deadlocks between the parallel group and other processes in
the system will be detected. When a process in the group requests a
lock that conflicts with one already held by another group member, but
not with any held by a process outside the group, grant it anyway, and
without waiting behind pending conflicting lock requests. At the end
of parallelism, transfer any locks held by workers and not released
back to the leader. In a world where lock requests within a group
can't conflict, there's no such thing as a deadlock within a parallel
group, so we don't need to detect deadlocks of that type, and there
will never be any unprincipled deadlocks within a group because there
will never be any deadlocks within a group at all.
The biggest problem with this approach is figuring out how to make it
safe. I originally proposed this approach on the "group locking"
thread and it kind of went down in flames. If two parallel backends
are trying to extend the same relation at the same time, or lock the
same tuple or page at the same time, those requests have got to
conflict. In those cases, we are using locking to enforce real mutual
exclusion. However, that doesn't seem like a serious problem. First,
locks of those types will not be held at the start of parallelism;
starting parallelism while you hold a lock of one of those types would
be ridiculous. Second, locks of those types are never long-lived: you
take them, do what you need to do, and then release them. Finally, I
don't believe you ever take any other heavyweight locks while holding
them; I *think* they're always the last heavyweight lock you take. So
I think we could allow these locks to conflict normally without
introducing any deadlock risk. The only locks that we need to
consider as mutually non-conflicting are relation and database object
locks. Those are a different kettle of fish: when we change a
relation's relfilenode, for example, we must keep the relation locked
for the duration of the transaction because of MVCC concenrs. The new
pg_class tuple isn't visible to anyone else yet, and any further
modifications to the relation must be done using the new relfilenode.
But none of that matters for a parallel worker, which shares the same
snapshot. There are still cases that do matter; for example, if a
parallel backend could REINDEX a backend being concurrently scanned by
another parallel backend in the same group, that would cause problems,
because REINDEX uses backend-local state that wouldn't be shared. But
these cases can also arise without parallelism, due to cursors, and
there is an existing mechanism (CheckTableNotInUse) to prevent them.
So I think that's OK, too. If not, we can fix other problems as we
find them; I don't see any major architectural problems here.
The other big problem with this approach is figuring out how to
implement it. It doesn't work to have the worker processes manipulate
the leader's lock-related data structures as if they were the parent,
both because the locking code relies on the shared data structures to
match its backend-private data structures and also because if we do
block waiting for a lock, there could be more than one waiting process
in the lock group at a time, and we need to know which specific
processes are waiting so we can wake them up at the correct time.
Instead, I think the way to do this is to add an additional field to
each PROCLOCK storing a pointer to the leader's PGPROC;
FindLockCycleRecurse() can check whether two PGPROCs have the same
leader pointer instead of whether the PGPROCs themselves are the same.
It can also refuse to follow a waits-for edge if that edge is of one
of the types we do want to conflict within a locking group (e.g.
relation extension).
---
I'm OK with any of these approaches if we can agree on how to solve
the problems they respectively present. Currently, I've got an
implementation of D-2; I started with an implementation of D-3, which
was flawed, but perhaps the idea is salveable, per the discussion
above. Andres has consistently advocated for D-1, but see the link
above for where I'm stuck in terms of implementing that. There may be
another approach we haven't come up with yet, too, for all I know, and
that's fine too if it accomplishes the goals listed above.
Thanks,
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
Attachments:
parallel-mode-v10.patchbinary/octet-stream; name=parallel-mode-v10.patchDownload
From a32da02168441eaec976895b49ac2f3d4d1ebda2 Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Fri, 30 Jan 2015 08:39:21 -0500
Subject: [PATCH 1/2] Create an infrastructure for parallel computation in
PostgreSQL.
This does four basic things. First, it provides convenience routines
to coordinate the startup and shutdown of parallel workers. Second,
it synchronizes various pieces of state (e.g. GUCs, combo CID
mappings, transaction snapshot) from the parallel group leader to the
worker processes. Third, it prohibits various operations that would
result in unsafe changes to that state while parallelism is active.
Finally, it propagates events that would result in an ErrorResponse,
NoticeResponse, or NotifyResponse message being sent to the client
from the parallel workers back to the master, from which they can then
be sent on to the client.
Robert Haas, Amit Kapila, Noah Misch, Rushabh Lathia, Jeevan Chalke.
Suggestions and review from Andres Freund, Heikki Linnakangas, Noah
Misch, Simon Riggs, Euler Taveira, and Jim Nasby.
---
contrib/postgres_fdw/connection.c | 3 +
src/backend/access/heap/heapam.c | 55 ++
src/backend/access/transam/Makefile | 2 +-
src/backend/access/transam/README.parallel | 223 ++++++
src/backend/access/transam/parallel.c | 1007 ++++++++++++++++++++++++++++
src/backend/access/transam/varsup.c | 7 +
src/backend/access/transam/xact.c | 486 +++++++++++++-
src/backend/access/transam/xlog.c | 8 +
src/backend/catalog/namespace.c | 11 +-
src/backend/commands/copy.c | 3 +-
src/backend/commands/sequence.c | 14 +
src/backend/executor/execMain.c | 30 +-
src/backend/executor/functions.c | 3 +
src/backend/executor/spi.c | 32 +-
src/backend/libpq/pqmq.c | 33 +-
src/backend/postmaster/bgworker.c | 50 ++
src/backend/storage/ipc/procarray.c | 44 ++
src/backend/storage/ipc/procsignal.c | 4 +
src/backend/storage/lmgr/predicate.c | 8 +
src/backend/tcop/postgres.c | 4 +-
src/backend/tcop/utility.c | 30 +-
src/backend/utils/adt/lockfuncs.c | 30 +
src/backend/utils/fmgr/dfmgr.c | 54 ++
src/backend/utils/misc/guc.c | 23 +
src/backend/utils/time/combocid.c | 74 ++
src/backend/utils/time/snapmgr.c | 210 +++++-
src/include/access/parallel.h | 68 ++
src/include/access/xact.h | 11 +
src/include/catalog/namespace.h | 2 +-
src/include/fmgr.h | 3 +
src/include/libpq/pqmq.h | 1 +
src/include/miscadmin.h | 1 +
src/include/postmaster/bgworker.h | 2 +
src/include/storage/procarray.h | 1 +
src/include/storage/procsignal.h | 1 +
src/include/utils/combocid.h | 3 +
src/include/utils/snapmgr.h | 5 +
37 files changed, 2499 insertions(+), 47 deletions(-)
create mode 100644 src/backend/access/transam/README.parallel
create mode 100644 src/backend/access/transam/parallel.c
create mode 100644 src/include/access/parallel.h
diff --git a/contrib/postgres_fdw/connection.c b/contrib/postgres_fdw/connection.c
index 4e02cb2..1a1e5b5 100644
--- a/contrib/postgres_fdw/connection.c
+++ b/contrib/postgres_fdw/connection.c
@@ -546,6 +546,7 @@ pgfdw_xact_callback(XactEvent event, void *arg)
switch (event)
{
+ case XACT_EVENT_PARALLEL_PRE_COMMIT:
case XACT_EVENT_PRE_COMMIT:
/* Commit all remote transactions during pre-commit */
do_sql_command(entry->conn, "COMMIT TRANSACTION");
@@ -588,11 +589,13 @@ pgfdw_xact_callback(XactEvent event, void *arg)
(errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
errmsg("cannot prepare a transaction that modified remote tables")));
break;
+ case XACT_EVENT_PARALLEL_COMMIT:
case XACT_EVENT_COMMIT:
case XACT_EVENT_PREPARE:
/* Pre-commit should have closed the open transaction */
elog(ERROR, "missed cleaning up connection during pre-commit");
break;
+ case XACT_EVENT_PARALLEL_ABORT:
case XACT_EVENT_ABORT:
/* Assume we might have lost track of prepared statements */
entry->have_error = true;
diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index 457cd70..16d8c59 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -42,6 +42,7 @@
#include "access/heapam_xlog.h"
#include "access/hio.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/relscan.h"
#include "access/sysattr.h"
#include "access/transam.h"
@@ -1051,7 +1052,13 @@ relation_open(Oid relationId, LOCKMODE lockmode)
/* Make note that we've accessed a temporary relation */
if (RelationUsesLocalBuffers(r))
+ {
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot access temporary tables during a parallel operation")));
MyXactAccessedTempRel = true;
+ }
pgstat_initstats(r);
@@ -1097,7 +1104,13 @@ try_relation_open(Oid relationId, LOCKMODE lockmode)
/* Make note that we've accessed a temporary relation */
if (RelationUsesLocalBuffers(r))
+ {
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot access temporary tables during a parallel operation")));
MyXactAccessedTempRel = true;
+ }
pgstat_initstats(r);
@@ -2234,6 +2247,17 @@ static HeapTuple
heap_prepare_insert(Relation relation, HeapTuple tup, TransactionId xid,
CommandId cid, int options)
{
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike heap_update() and heap_delete(), an insert should never create
+ * a combo CID, so it might be possible to relax this restriction, but
+ * not without more thought and testing.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot insert tuples during a parallel operation")));
+
if (relation->rd_rel->relhasoids)
{
#ifdef NOT_USED
@@ -2641,6 +2665,16 @@ heap_delete(Relation relation, ItemPointer tid,
Assert(ItemPointerIsValid(tid));
+ /*
+ * Forbid this during a parallel operation, lets it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot delete tuples during a parallel operation")));
+
block = ItemPointerGetBlockNumber(tid);
buffer = ReadBuffer(relation, block);
page = BufferGetPage(buffer);
@@ -3090,6 +3124,16 @@ heap_update(Relation relation, ItemPointer otid, HeapTuple newtup,
Assert(ItemPointerIsValid(otid));
/*
+ * Forbid this during a parallel operation, lets it allocate a combocid.
+ * Other workers might need that combocid for visibility checks, and we
+ * have no provision for broadcasting it to them.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
+ /*
* Fetch the list of attributes to be checked for HOT update. This is
* wasted effort if we fail to update or have to put the new tuple on a
* different page. But we must compute the list before obtaining buffer
@@ -5388,6 +5432,17 @@ heap_inplace_update(Relation relation, HeapTuple tuple)
uint32 oldlen;
uint32 newlen;
+ /*
+ * For now, parallel operations are required to be strictly read-only.
+ * Unlike a regular update, this should never create a combo CID, so it
+ * might be possible to relax this restriction, but not without more
+ * thought and testing. It's not clear that it would be useful, anyway.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot update tuples during a parallel operation")));
+
buffer = ReadBuffer(relation, ItemPointerGetBlockNumber(&(tuple->t_self)));
LockBuffer(buffer, BUFFER_LOCK_EXCLUSIVE);
page = (Page) BufferGetPage(buffer);
diff --git a/src/backend/access/transam/Makefile b/src/backend/access/transam/Makefile
index 9d4d5db..94455b2 100644
--- a/src/backend/access/transam/Makefile
+++ b/src/backend/access/transam/Makefile
@@ -12,7 +12,7 @@ subdir = src/backend/access/transam
top_builddir = ../../../..
include $(top_builddir)/src/Makefile.global
-OBJS = clog.o commit_ts.o multixact.o rmgr.o slru.o subtrans.o \
+OBJS = clog.o commit_ts.o multixact.o parallel.o rmgr.o slru.o subtrans.o \
timeline.o transam.o twophase.o twophase_rmgr.o varsup.o \
xact.o xlog.o xlogarchive.o xlogfuncs.o \
xloginsert.o xlogreader.o xlogutils.o
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
new file mode 100644
index 0000000..1005186
--- /dev/null
+++ b/src/backend/access/transam/README.parallel
@@ -0,0 +1,223 @@
+Overview
+========
+
+PostgreSQL provides some simple facilities to make writing parallel algorithms
+easier. Using a data structure called a ParallelContext, you can arrange to
+launch background worker processes, initialize their state to match that of
+the backend which initiated parallelism, communicate with them via dynamic
+shared memory, and write reasonably complex code that can run either in the
+user backend or in one of the parallel workers without needing to be aware of
+where it's running.
+
+The backend which starts a parallel operation (hereafter, the initiating
+backend) starts by creating a dynamic shared memory segment which will last
+for the lifetime of the parallel operation. This dynamic shared memory segment
+will contain (1) a shm_mq that can be used to transport errors (and other
+messages reported via elog/ereport) from the worker back to the initiating
+backend; (2) serialized representations of the initiating backend's private
+state, so that the worker can synchronize its state with of the initiating
+backend; and (3) any other data structures which a particular user of the
+ParallelContext data structure may wish to add for its own purposes. Once
+the initiating backend has initialized the dynamic shared memory segment, it
+asks the postmaster to launch the appropriate number of parallel workers.
+These workers then connect to the dynamic shared memory segment, initiate
+their state, and then invoke the appropriate entrypoint, as further detailed
+below.
+
+Error Reporting
+===============
+
+When started, each parallel worker begins by attaching the dynamic shared
+memory segment and locating the shm_mq to be used for error reporting; it
+redirects all of its protocol messages to this shm_mq. Prior to this point,
+any failure of the background worker will not be reported to the initiating
+backend; from the point of view of the initiating backend, the worker simply
+failed to start. The initiating backend must anyway be prepared to cope
+with fewer parallel workers than it originally requested, so catering to
+this case imposes no additional burden.
+
+Whenever a new message (or partial message; very large messages may wrap) is
+sent to the error-reporting queue, PROCSIG_PARALLEL_MESSAGE is sent to the
+initiating backend. This causes the next CHECK_FOR_INTERRUPTS() in the
+initiating backend to read and rethrow the message. For the most part, this
+makes error reporting in parallel mode "just work". Of course, to work
+properly, it is important that the code the initiating backend is executing
+CHECK_FOR_INTERRUPTS() regularly and avoid blocking interrupt processing for
+long periods of time, but those are good things to do anyway.
+
+(A currently-unsolved problem is that some messages may get written to the
+system log twice, once in the backend where the report was originally
+generated, and again when the initiating backend rethrows the message. If
+we decide to suppress one of these reports, it should probably be second one;
+otherwise, if the worker is for some reason unable to propagate the message
+back to the initiating backend, the message will be lost altogether.)
+
+State Sharing
+=============
+
+It's possible to write C code which works correctly without parallelism, but
+which fails when parallelism is used. No parallel infrastructure can
+completely eliminate this problem, because any global variable is a risk.
+There's no general mechanism for ensuring that every global variable in the
+worker will have the same value that it does in the initiating backend; even
+if we could ensure that, some function we're calling could update the variable
+after each call, and only the backend where that update is performed will see
+the new value. Similar problems can arise with any more-complex data
+structure we might choose to use. For example, a pseudo-random number
+generator should, given a particular seed value, produce the same predictable
+series of values every time. But it does this by relying on some private
+state which won't automatically be shared between cooperating backends. A
+parallel-safe PRNG would need to store its state in dynamic shared memory, and
+would require locking. The parallelism infrastructure has no way of knowing
+whether the user intends to call code that has this sort of problem, and can't
+do anything about it anyway.
+
+Instead, we take a more pragmatic approach. First, we try to make as many of
+the operations that are safe outside of parallel mode work correctly in
+parallel mode as well. Second, we try to prohibit common unsafe operations
+via suitable error checks. These checks are intended to catch 100% of
+unsafe things that a user might do from the SQL interface, but code written
+in C can do unsafe things that won't trigger these checks. The error checks
+are engaged via EnterParallelMode(), which should be called before creating
+a parallel context, and disarmed via ExitParallelMode(), which should be
+called after all parallel contexts have been destroyed. The most
+significant restriction imposed by parallel mode is that all operations must
+be strictly read-only; we allow no writes to the database and no DDL. We
+might try to relax these restrictions in the future.
+
+To make as many operations as possible safe in parallel mode, we try to copy
+the most important pieces of state from the initiating backend to each parallel
+worker. This includes:
+
+ - The set of libraries dynamically loaded by dfmgr.c.
+
+ - The authenticated user ID and current database. Each parallel worker
+ will connect to the same database as the initiating backend, using the
+ same user ID.
+
+ - The values of all GUCs. Accordingly, permanent changes to the value of
+ any GUC are forbidden while in parallel mode; but temporary changes,
+ such as entering a function with non-NULL proconfig, are OK.
+
+ - The current subtransaction's XID, the top-level transaction's XID, and
+ the list of XIDs considered current (that is, they are in-progress or
+ subcommitted). This information is needed to ensure that tuple visibility
+ checks return the same results in the worker as they do in the
+ initiating backend. See also the section Transaction Integration, below.
+
+ - The combo CID mappings. This is needed to ensure consistent answers to
+ tuple visibility checks. The need to synchronize this data structure is
+ a major reason why we can't support writes in parallel mode: such writes
+ might create new combo CIDs, and we have no way to let other workers
+ (or the initiating backend) know about them.
+
+ - The transaction snapshot.
+
+ - The active snapshot, which might be different from the transaction
+ snapshot.
+
+ - The currently active user ID and security context. Note that this is
+ the fourth user ID we restore: the initial step of binding to the correct
+ database also involves restoring the authenticated user ID. When GUC
+ values are restored, this incidentally sets SessionUserId and OuterUserId
+ to the correct values. This final step restores CurrentUserId.
+
+To prevent undetected or unprincipled deadlocks when running in parallel mode,
+this could should eventually handle heavyweight locks in some way. This is
+not implemented yet.
+
+Transaction Integration
+=======================
+
+Regardless of what the TransactionState stack looks like in the parallel
+leader, each parallel worker ends up with a stack of depth 1. This stack
+entry is marked with the special transaction block state
+TBLOCK_PARALLEL_INPROGRESS so that it's not confused with an ordinary
+toplevel transaction. The XID of this TransactionState is set to the XID of
+the innermost currently-active subtransaction in the initiating backend. The
+initiating backend's toplevel XID, and the XIDs of all current (in-progress
+or subcommitted) XIDs are stored separately from the TransactionState stack,
+but in such a way that GetTopTransactionId(), GetTopTransactionIdIfAny(), and
+TransactionIdIsCurrentTransactionId() return the same values that they would
+in the initiating backend. We could copy the entire transaction state stack,
+but most of it would be useless: for example, you can't roll back to a
+savepoint from within a parallel worker, and there are no resources to
+associated with the memory contexts or resource owners of intermediate
+subtransactions.
+
+No meaningful change to the transaction state can be made while in parallel
+mode. No XIDs can be assigned, and no subtransactions can start or end,
+because we have no way of communicating these state changes to cooperating
+backends, or of synchronizing them. It's clearly unworkable for the initiating
+backend to exit any transaction or subtransaction that was in progress when
+parallelism was started before all parallel workers have exited; and it's even
+more clearly crazy for a parallel worker to try to subcommit or subabort the
+current subtransaction and execute in some other transaction context than was
+present in the initiating backend. It might be practical to allow internal
+sub-transactions (e.g. to implement a PL/pgsql EXCEPTION block) to be used in
+parallel mode, provided that they are XID-less, because other backends
+wouldn't really need to know about those transactions or do anything
+differently because of them. Right now, we don't even allow that.
+
+At the end of a parallel operation, which can happen either because it
+completed successfully or because it was interrupted by an error, parallel
+workers associated with that operation exit. In the error case, transaction
+abort processing in the parallel leader kills of any remaining workers, and
+the parallel leader then waits for them to die. In the case of a successful
+parallel operation, the parallel leader does not send any signals, but must
+wait for workers to complete and exit of their own volition. In either
+case, it is very important that all workers actually exit before the
+parallel leader cleans up the (sub)transaction in which they were created;
+otherwise, chaos can ensue. For example, if the leader is rolling back the
+transaction that created the relation being scanned by a worker, the
+relation could disappear while the worker is still busy scanning it. That's
+not safe.
+
+Generally, the cleanup performed by each worker at this point is similar to
+top-level commit or abort. Each backend has its own resource owners: buffer
+pins, catcache or relcache reference counts, tuple descriptors, and so on
+are managed separately by each backend, and must free them before exiting.
+There are, however, some important differences between parallel worker
+commit or abort and a real top-level transaction commit or abort. Most
+importantly:
+
+ - No commit or abort record is written; the initiating backend is
+ responsible for this.
+
+ - Cleanup of pg_temp namespaces is not done. Parallel workers cannot
+ safely access the initiating backend's pg_temp namespace, and should
+ not create one of their own.
+
+Coding Conventions
+===================
+
+Before beginning any parallel operation, call EnterParallelMode(); after all
+parallel operations are completed, call ExitParallelMode(). To actually
+parallelize a particular operation, use a ParallelContext. The basic coding
+pattern looks like this:
+
+ EnterParallelMode(); /* prohibit unsafe state changes */
+
+ pcxt = CreateParallelContext(entrypoint, nworkers);
+
+ /* Allow space for application-specific data here. */
+ shm_toc_estimate_chunk(&pcxt->estimator, size);
+ shm_toc_estimate_keys(&pcxt->estimator, keys);
+
+ InitializeParallelDSM(pcxt); /* create DSM and copy state to it */
+
+ /* Store the data for which we reserved space. */
+ space = shm_toc_allocate(pcxt->toc, size);
+ shm_toc_insert(pcxt->toc, key, space);
+
+ LaunchParallelWorkers(pcxt);
+
+ /* do parallel stuff */
+
+ WaitForParallelWorkersToFinish(pcxt);
+
+ /* read any final results from dynamic shared memory */
+
+ DestroyParallelContext(pcxt);
+
+ ExitParallelMode();
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
new file mode 100644
index 0000000..8ed7314
--- /dev/null
+++ b/src/backend/access/transam/parallel.c
@@ -0,0 +1,1007 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.c
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * IDENTIFICATION
+ * src/backend/access/transam/parallel.c
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#include "postgres.h"
+
+#include "access/xact.h"
+#include "access/xlog.h"
+#include "access/parallel.h"
+#include "commands/async.h"
+#include "libpq/libpq.h"
+#include "libpq/pqformat.h"
+#include "libpq/pqmq.h"
+#include "miscadmin.h"
+#include "storage/ipc.h"
+#include "storage/sinval.h"
+#include "storage/spin.h"
+#include "tcop/tcopprot.h"
+#include "utils/combocid.h"
+#include "utils/guc.h"
+#include "utils/memutils.h"
+#include "utils/resowner.h"
+#include "utils/snapmgr.h"
+
+/*
+ * We don't want to waste a lot of memory on an error queue which, most of
+ * the time, will process only a handful of small messages. However, it is
+ * desirable to make it large enough that a typical ErrorResponse can be sent
+ * without blocking. That way, a worker that errors out can write the whole
+ * message into the queue and terminate without waiting for the user backend.
+ */
+#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_LIBRARY UINT64CONST(0xFFFFFFFFFFFF0003)
+#define PARALLEL_KEY_GUC UINT64CONST(0xFFFFFFFFFFFF0004)
+#define PARALLEL_KEY_COMBO_CID UINT64CONST(0xFFFFFFFFFFFF0005)
+#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
+#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
+#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0009)
+
+/* 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;
+
+ /* Mutex protects remaining fields. */
+ slock_t mutex;
+
+ /* Track whether workers have attached. */
+ int workers_expected;
+ int workers_attached;
+
+ /* Maximum XactLastRecEnd of any worker. */
+ XLogRecPtr last_xlog_end;
+} 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
+ * and < the number of workers before any user code is invoked; each parallel
+ * worker will get a different parallel worker number.
+ */
+int ParallelWorkerNumber = -1;
+
+/* Is there a parallel message pending which we need to receive? */
+bool ParallelMessagePending = false;
+
+/* Pointer to our fixed parallel state. */
+static FixedParallelState *MyFixedParallelState;
+
+/* List of active parallel contexts. */
+static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
+
+/* Private functions. */
+static void HandleParallelMessage(ParallelContext *, int, StringInfo msg);
+static void ParallelErrorContext(void *arg);
+static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
+static void ParallelWorkerMain(Datum main_arg);
+
+/*
+ * Establish a new parallel context. This should be done after entering
+ * parallel mode, and (unless there is an error) the context should be
+ * destroyed before exiting the current subtransaction.
+ */
+ParallelContext *
+CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* It is unsafe to create a parallel context if not in parallel mode. */
+ Assert(IsInParallelMode());
+
+ /* Number of workers should be non-negative. */
+ Assert(nworkers >= 0);
+
+ /*
+ * If dynamic shared memory is not available, we won't be able to use
+ * background workers.
+ */
+ if (dynamic_shared_memory_type == DSM_IMPL_NONE)
+ nworkers = 0;
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Initialize a new ParallelContext. */
+ pcxt = palloc0(sizeof(ParallelContext));
+ pcxt->subid = GetCurrentSubTransactionId();
+ pcxt->nworkers = nworkers;
+ pcxt->entrypoint = entrypoint;
+ pcxt->error_context_stack = error_context_stack;
+ shm_toc_initialize_estimator(&pcxt->estimator);
+ dlist_push_head(&pcxt_list, &pcxt->node);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish a new parallel context that calls a function provided by an
+ * extension. This works around the fact that the library might get mapped
+ * at a different address in each backend.
+ */
+ParallelContext *
+CreateParallelContextForExternalFunction(char *library_name,
+ char *function_name,
+ int nworkers)
+{
+ MemoryContext oldcontext;
+ ParallelContext *pcxt;
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Create the context. */
+ pcxt = CreateParallelContext(ParallelExtensionTrampoline, nworkers);
+ pcxt->library_name = pstrdup(library_name);
+ pcxt->function_name = pstrdup(function_name);
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+
+ return pcxt;
+}
+
+/*
+ * Establish the dynamic shared memory segment for a parallel context and
+ * copied state and other bookkeeping information that will need by parallel
+ * workers into it.
+ */
+void
+InitializeParallelDSM(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ Size library_len = 0;
+ Size guc_len = 0;
+ Size combocidlen = 0;
+ Size tsnaplen = 0;
+ Size asnaplen = 0;
+ Size tstatelen = 0;
+ Size segsize = 0;
+ int i;
+ FixedParallelState *fps;
+ Snapshot transaction_snapshot = GetTransactionSnapshot();
+ Snapshot active_snapshot = GetActiveSnapshot();
+
+ /* We might be running in a very short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Allow space to store the fixed-size parallel state. */
+ shm_toc_estimate_chunk(&pcxt->estimator, sizeof(FixedParallelState));
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /*
+ * Normally, the user will have requested at least one worker process,
+ * but if by chance they have not, we can skip a bunch of things here.
+ */
+ if (pcxt->nworkers > 0)
+ {
+ /* Estimate space for various kinds of state sharing. */
+ library_len = EstimateLibraryStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, library_len);
+ guc_len = EstimateGUCStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, guc_len);
+ combocidlen = EstimateComboCIDStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, combocidlen);
+ tsnaplen = EstimateSnapshotSpace(transaction_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, tsnaplen);
+ asnaplen = EstimateSnapshotSpace(active_snapshot);
+ shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
+ tstatelen = EstimateTransactionStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ /* If you add more chunks here, you probably need to add keys. */
+ shm_toc_estimate_keys(&pcxt->estimator, 6);
+
+ /* Estimate space need for error queues. */
+ StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
+ PARALLEL_ERROR_QUEUE_SIZE,
+ "parallel error queue size not buffer-aligned");
+ shm_toc_estimate_chunk(&pcxt->estimator,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+
+ /* Estimate how much we'll need for extension entrypoint info. */
+ if (pcxt->library_name != NULL)
+ {
+ Assert(pcxt->entrypoint == ParallelExtensionTrampoline);
+ Assert(pcxt->function_name != NULL);
+ shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name)
+ + strlen(pcxt->function_name) + 2);
+ shm_toc_estimate_keys(&pcxt->estimator, 1);
+ }
+ }
+
+ /*
+ * Create DSM and initialize with new table of contents. But if the user
+ * didn't request any workers, then don't bother creating a dynamic shared
+ * memory segment; instead, just use backend-private memory.
+ *
+ * Also, if we can't create a dynamic shared memory segment because the
+ * maximum number of segments have already been created, then fall back
+ * to backend-private memory, and plan not to use any workers. We hope
+ * this won't happen very often, but it's better to abandon the use of
+ * parallelism than to fail outright.
+ */
+ segsize = shm_toc_estimate(&pcxt->estimator);
+ if (pcxt->nworkers != 0)
+ pcxt->seg = dsm_create(segsize, DSM_CREATE_NULL_IF_MAXSEGMENTS);
+ if (pcxt->seg != NULL)
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC,
+ dsm_segment_address(pcxt->seg),
+ segsize);
+ else
+ {
+ pcxt->nworkers = 0;
+ pcxt->private = MemoryContextAlloc(TopMemoryContext, segsize);
+ pcxt->toc = shm_toc_create(PARALLEL_MAGIC, pcxt->private, segsize);
+ }
+
+ /* Initialize fixed-size state in shared memory. */
+ fps = (FixedParallelState *)
+ shm_toc_allocate(pcxt->toc, sizeof(FixedParallelState));
+ fps->database_id = MyDatabaseId;
+ fps->authenticated_user_id = GetAuthenticatedUserId();
+ GetUserIdAndSecContext(&fps->current_user_id, &fps->sec_context);
+ fps->parallel_master_pgproc = MyProc;
+ fps->parallel_master_pid = MyProcPid;
+ fps->parallel_master_backend_id = MyBackendId;
+ fps->entrypoint = pcxt->entrypoint;
+ SpinLockInit(&fps->mutex);
+ fps->workers_expected = pcxt->nworkers;
+ fps->workers_attached = 0;
+ fps->last_xlog_end = 0;
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
+
+ /* We can skip the rest of this if we're not budgeting for any workers. */
+ if (pcxt->nworkers > 0)
+ {
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ char *error_queue_space;
+
+ /* Serialize shared libraries we have loaded. */
+ libraryspace = shm_toc_allocate(pcxt->toc, library_len);
+ SerializeLibraryState(library_len, libraryspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LIBRARY, libraryspace);
+
+ /* Serialize GUC settings. */
+ gucspace = shm_toc_allocate(pcxt->toc, guc_len);
+ SerializeGUCState(guc_len, gucspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_GUC, gucspace);
+
+ /* Serialize combo CID state. */
+ combocidspace = shm_toc_allocate(pcxt->toc, combocidlen);
+ SerializeComboCIDState(combocidlen, combocidspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_COMBO_CID, combocidspace);
+
+ /* Serialize transaction snapshot and active snapshot. */
+ tsnapspace = shm_toc_allocate(pcxt->toc, tsnaplen);
+ SerializeSnapshot(transaction_snapshot, tsnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT,
+ tsnapspace);
+ asnapspace = shm_toc_allocate(pcxt->toc, asnaplen);
+ SerializeSnapshot(active_snapshot, asnapspace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ACTIVE_SNAPSHOT, asnapspace);
+
+ /* Serialize transaction state. */
+ tstatespace = shm_toc_allocate(pcxt->toc, tstatelen);
+ SerializeTransactionState(tstatelen, tstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+
+ /* Allocate space for worker information. */
+ pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
+
+ /*
+ * Establish error queues in dynamic shared memory.
+ *
+ * These queues should be used only for transmitting ErrorResponse,
+ * NoticeResponse, and NotifyResponse protocol messages. Tuple data
+ * should be transmitted via separate (possibly larger?) queues.
+ */
+ error_queue_space =
+ shm_toc_allocate(pcxt->toc,
+ PARALLEL_ERROR_QUEUE_SIZE * pcxt->nworkers);
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ char *start;
+ shm_mq *mq;
+
+ start = error_queue_space + i * PARALLEL_ERROR_QUEUE_SIZE;
+ mq = shm_mq_create(start, PARALLEL_ERROR_QUEUE_SIZE);
+ shm_mq_set_receiver(mq, MyProc);
+ pcxt->worker[i].error_mqh = shm_mq_attach(mq, pcxt->seg, NULL);
+ }
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
+
+ /* Serialize extension entrypoint information. */
+ if (pcxt->library_name != NULL)
+ {
+ Size lnamelen = strlen(pcxt->library_name);
+ char *extensionstate;
+
+ extensionstate = shm_toc_allocate(pcxt->toc, lnamelen
+ + strlen(pcxt->function_name) + 2);
+ strcpy(extensionstate, pcxt->library_name);
+ strcpy(extensionstate + lnamelen + 1, pcxt->function_name);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE,
+ extensionstate);
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Launch parallel workers.
+ */
+void
+LaunchParallelWorkers(ParallelContext *pcxt)
+{
+ MemoryContext oldcontext;
+ BackgroundWorker worker;
+ int i;
+ bool any_registrations_failed = false;
+
+ /* Skip this if we have no workers. */
+ if (pcxt->nworkers == 0)
+ return;
+
+ /* If we do have workers, we'd better have a DSM segment. */
+ Assert(pcxt->seg != NULL);
+
+ /* We might be running in a short-lived memory context. */
+ oldcontext = MemoryContextSwitchTo(TopTransactionContext);
+
+ /* Configure a worker. */
+ snprintf(worker.bgw_name, BGW_MAXLEN, "parallel worker for PID %d",
+ MyProcPid);
+ 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 = ParallelWorkerMain;
+ worker.bgw_main_arg = UInt32GetDatum(dsm_segment_handle(pcxt->seg));
+ worker.bgw_notify_pid = MyProcPid;
+
+ /*
+ * Start workers.
+ *
+ * The caller must be able to tolerate ending up with fewer workers than
+ * expected, so there is no need to throw an error here if registration
+ * fails. It wouldn't help much anyway, because registering the worker
+ * in no way guarantees that it will start up and initialize successfully.
+ */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (!any_registrations_failed &&
+ RegisterDynamicBackgroundWorker(&worker,
+ &pcxt->worker[i].bgwhandle))
+ shm_mq_set_handle(pcxt->worker[i].error_mqh,
+ pcxt->worker[i].bgwhandle);
+ else
+ {
+ /*
+ * If we weren't able to register the worker, then we've bumped
+ * up against the max_worker_processes limit, and future
+ * registrations will probably fail too, so arrange to skip them.
+ * But we still have to execute this code for the remaining slots
+ * to make sure that we forget about the error queues we budgeted
+ * for those workers. Otherwise, we'll wait for them to start,
+ * but they never will.
+ */
+ any_registrations_failed = true;
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ }
+ }
+
+ /* Restore previous memory context. */
+ MemoryContextSwitchTo(oldcontext);
+}
+
+/*
+ * Wait for all workers to exit.
+ *
+ * Even if the parallel operation seems to have completed successfully, it's
+ * important to call this function afterwards. We must not miss any errors
+ * the workers may have thrown during the parallel operation, or any that they
+ * may yet throw while shutting down.
+ *
+ * Also, we want to update our notion of XactLastRecEnd based on worker
+ * feedback.
+ */
+void
+WaitForParallelWorkersToFinish(ParallelContext *pcxt)
+{
+ for (;;)
+ {
+ bool anyone_alive = false;
+ int i;
+
+ /*
+ * This will process any parallel messages that are pending, which
+ * may change the outcome of the loop that follows. It may also
+ * throw an error propagated from a worker.
+ */
+ CHECK_FOR_INTERRUPTS();
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ anyone_alive = true;
+ break;
+ }
+ }
+
+ if (!anyone_alive)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, -1);
+ ResetLatch(&MyProc->procLatch);
+ }
+
+ if (pcxt->toc != NULL)
+ {
+ FixedParallelState *fps;
+
+ fps = shm_toc_lookup(pcxt->toc, PARALLEL_KEY_FIXED);
+ if (fps->last_xlog_end > XactLastRecEnd)
+ XactLastRecEnd = fps->last_xlog_end;
+ }
+}
+
+/*
+ * Destroy a parallel context.
+ *
+ * If expecting a clean exit, you should use WaitForParallelWorkersToFinish()
+ * first, before calling this function. When this function is invoked, any
+ * remaining workers are forcibly killed; the dynamic shared memory segment
+ * is unmapped; and we then wait (uninterruptibly) for the workers to exit.
+ */
+void
+DestroyParallelContext(ParallelContext *pcxt)
+{
+ int i;
+
+ /*
+ * Be careful about order of operations here! We remove the parallel
+ * context from the list before we do anything else; otherwise, if an
+ * error occurs during a subsequent step, we might try to nuke it again
+ * from AtEOXact_Parallel or AtEOSubXact_Parallel.
+ */
+ dlist_delete(&pcxt->node);
+
+ /* Kill each worker in turn, and forget their error queues. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ if (pcxt->worker[i].bgwhandle != NULL)
+ TerminateBackgroundWorker(pcxt->worker[i].bgwhandle);
+ if (pcxt->worker[i].error_mqh != NULL)
+ {
+ pfree(pcxt->worker[i].error_mqh);
+ 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 (pcxt->seg != NULL)
+ {
+ dsm_detach(pcxt->seg);
+ pcxt->seg = NULL;
+ }
+
+ /*
+ * If this parallel context is actually in backend-private memory rather
+ * than shared memory, free that memory instead.
+ */
+ if (pcxt->private != NULL)
+ {
+ pfree(pcxt->private);
+ pcxt->private = NULL;
+ }
+
+ /* Wait until the workers actually die. */
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ BgwHandleStatus status;
+
+ if (pcxt->worker[i].bgwhandle == NULL)
+ continue;
+
+ /*
+ * We can't finish transaction commit or abort until all of the
+ * workers are dead. This means, in particular, that we can't respond
+ * to interrupts at this stage.
+ */
+ HOLD_INTERRUPTS();
+ status = WaitForBackgroundWorkerShutdown(pcxt->worker[i].bgwhandle);
+ RESUME_INTERRUPTS();
+
+ /*
+ * If the postmaster kicked the bucket, we have no chance of cleaning
+ * up safely -- we won't be able to tell when our workers are actually
+ * dead. This doesn't necessitate a PANIC since they will all abort
+ * eventually, but we can't safely continue this session.
+ */
+ if (status == BGWH_POSTMASTER_DIED)
+ ereport(FATAL,
+ (errcode(ERRCODE_ADMIN_SHUTDOWN),
+ errmsg("postmaster exited during a parallel transaction")));
+
+ /* Release memory. */
+ pfree(pcxt->worker[i].bgwhandle);
+ pcxt->worker[i].bgwhandle = NULL;
+ }
+
+ /* Free the worker array itself. */
+ if (pcxt->worker != NULL)
+ {
+ pfree(pcxt->worker);
+ pcxt->worker = NULL;
+ }
+
+ /* Free memory. */
+ pfree(pcxt);
+}
+
+/*
+ * Are there any parallel contexts currently active?
+ */
+bool
+ParallelContextActive(void)
+{
+ return !dlist_is_empty(&pcxt_list);
+}
+
+/*
+ * Handle receipt of an interrupt indicating a parallel worker message.
+ */
+void
+HandleParallelMessageInterrupt(void)
+{
+ int save_errno = errno;
+
+ InterruptPending = true;
+ ParallelMessagePending = true;
+ SetLatch(MyLatch);
+
+ errno = save_errno;
+}
+
+/*
+ * Handle any queued protocol messages received from parallel workers.
+ */
+void
+HandleParallelMessages(void)
+{
+ dlist_iter iter;
+
+ ParallelMessagePending = false;
+
+ dlist_foreach(iter, &pcxt_list)
+ {
+ ParallelContext *pcxt;
+ int i;
+ Size nbytes;
+ void *data;
+
+ pcxt = dlist_container(ParallelContext, node, iter.cur);
+ if (pcxt->worker == NULL)
+ continue;
+
+ for (i = 0; i < pcxt->nworkers; ++i)
+ {
+ /*
+ * Read as many messages as we can from each worker, but stop
+ * when either (1) the error queue goes away, which can happen if
+ * we receive a Terminate message from the worker; or (2) no more
+ * messages can be read from the worker without blocking.
+ */
+ while (pcxt->worker[i].error_mqh != NULL)
+ {
+ shm_mq_result res;
+
+ res = shm_mq_receive(pcxt->worker[i].error_mqh, &nbytes,
+ &data, true);
+ if (res == SHM_MQ_WOULD_BLOCK)
+ break;
+ else if (res == SHM_MQ_SUCCESS)
+ {
+ StringInfoData msg;
+
+ initStringInfo(&msg);
+ appendBinaryStringInfo(&msg, data, nbytes);
+ HandleParallelMessage(pcxt, i, &msg);
+ pfree(msg.data);
+ }
+ else
+ ereport(ERROR,
+ (errcode(ERRCODE_INTERNAL_ERROR), /* XXX: wrong errcode? */
+ errmsg("lost connection to parallel worker")));
+
+ /* This might make the error queue go away. */
+ CHECK_FOR_INTERRUPTS();
+ }
+ }
+ }
+}
+
+/*
+ * Handle a single protocol message received from a single parallel worker.
+ */
+static void
+HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg)
+{
+ char msgtype;
+
+ msgtype = pq_getmsgbyte(msg);
+
+ switch (msgtype)
+ {
+ case 'K': /* BackendKeyData */
+ {
+ int32 pid = pq_getmsgint(msg, 4);
+ (void) pq_getmsgint(msg, 4); /* discard cancel key */
+ (void) pq_getmsgend(msg);
+ pcxt->worker[i].pid = pid;
+ break;
+ }
+
+ case 'E': /* ErrorResponse */
+ case 'N': /* NoticeResponse */
+ {
+ ErrorData edata;
+ ErrorContextCallback errctx;
+ ErrorContextCallback *save_error_context_stack;
+
+ /*
+ * Rethrow the error using the error context callbacks that
+ * were in effect when the context was created, not the
+ * current ones.
+ */
+ save_error_context_stack = error_context_stack;
+ errctx.callback = ParallelErrorContext;
+ errctx.arg = &pcxt->worker[i].pid;
+ errctx.previous = pcxt->error_context_stack;
+ error_context_stack = &errctx;
+
+ /* Parse ErrorReponse or NoticeResponse. */
+ pq_parse_errornotice(msg, &edata);
+
+ /* Death of a worker isn't enough justification for suicide. */
+ edata.elevel = Min(edata.elevel, ERROR);
+
+ /* Rethrow error or notice. */
+ ThrowErrorData(&edata);
+
+ /* Restore previous context. */
+ error_context_stack = save_error_context_stack;
+
+ break;
+ }
+
+ case 'A': /* NotifyResponse */
+ {
+ /* Propagate NotifyResponse. */
+ pq_putmessage(msg->data[0], &msg->data[1], msg->len - 1);
+ break;
+ }
+
+ case 'X': /* Terminate, indicating clean exit */
+ {
+ pfree(pcxt->worker[i].bgwhandle);
+ pfree(pcxt->worker[i].error_mqh);
+ pcxt->worker[i].bgwhandle = NULL;
+ pcxt->worker[i].error_mqh = NULL;
+ break;
+ }
+
+ default:
+ {
+ elog(ERROR, "unknown message type: %c (%d bytes)",
+ msgtype, msg->len);
+ }
+ }
+}
+
+/*
+ * End-of-subtransaction cleanup for parallel contexts.
+ *
+ * Currently, it's forbidden to enter or leave a subtransaction while
+ * parallel mode is in effect, so we could just blow away everything. But
+ * we may want to relax that restriction in the future, so this code
+ * contemplates that there may be multiple subtransaction IDs in pcxt_list.
+ */
+void
+AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (pcxt->subid != mySubId)
+ break;
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * End-of-transaction cleanup for parallel contexts.
+ */
+void
+AtEOXact_Parallel(bool isCommit)
+{
+ while (!dlist_is_empty(&pcxt_list))
+ {
+ ParallelContext *pcxt;
+
+ pcxt = dlist_head_element(ParallelContext, node, &pcxt_list);
+ if (isCommit)
+ elog(WARNING, "leaked parallel context");
+ DestroyParallelContext(pcxt);
+ }
+}
+
+/*
+ * Main entrypoint for parallel workers.
+ */
+static void
+ParallelWorkerMain(Datum main_arg)
+{
+ dsm_segment *seg;
+ shm_toc *toc;
+ FixedParallelState *fps;
+ char *error_queue_space;
+ shm_mq *mq;
+ shm_mq_handle *mqh;
+ char *libraryspace;
+ char *gucspace;
+ char *combocidspace;
+ char *tsnapspace;
+ char *asnapspace;
+ char *tstatespace;
+ StringInfoData msgbuf;
+
+ /* Establish signal handlers. */
+ pqsignal(SIGTERM, die);
+ BackgroundWorkerUnblockSignals();
+
+ /* Set up a memory context and resource owner. */
+ Assert(CurrentResourceOwner == NULL);
+ CurrentResourceOwner = ResourceOwnerCreate(NULL, "parallel toplevel");
+ CurrentMemoryContext = AllocSetContextCreate(TopMemoryContext,
+ "parallel worker",
+ ALLOCSET_DEFAULT_MINSIZE,
+ ALLOCSET_DEFAULT_INITSIZE,
+ ALLOCSET_DEFAULT_MAXSIZE);
+
+ /*
+ * 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(DatumGetUInt32(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(PARALLEL_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")));
+
+ /* Determine and set our worker number. */
+ fps = shm_toc_lookup(toc, PARALLEL_KEY_FIXED);
+ Assert(fps != NULL);
+ Assert(ParallelWorkerNumber == -1);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->workers_attached < fps->workers_expected)
+ ParallelWorkerNumber = fps->workers_attached++;
+ SpinLockRelease(&fps->mutex);
+ if (ParallelWorkerNumber < 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("too many parallel workers already attached")));
+ MyFixedParallelState = fps;
+
+ /*
+ * Now that we have a worker number, we can find and attach to the error
+ * queue provided for us. That's good, because until we do that, any
+ * errors that happen here will not be reported back to the process that
+ * requested that this worker be launched.
+ */
+ error_queue_space = shm_toc_lookup(toc, PARALLEL_KEY_ERROR_QUEUE);
+ mq = (shm_mq *) (error_queue_space +
+ 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_set_parallel_master(fps->parallel_master_pid,
+ fps->parallel_master_backend_id);
+
+ /*
+ * Send a BackendKeyData message to the process that initiated parallelism
+ * so that it has access to our PID before it receives any other messages
+ * from us. Our cancel key is sent, too, since that's the way the protocol
+ * message is defined, but it won't actually be used for anything in this
+ * case.
+ */
+ pq_beginmessage(&msgbuf, 'K');
+ pq_sendint(&msgbuf, (int32) MyProcPid, sizeof(int32));
+ pq_sendint(&msgbuf, (int32) MyCancelKey, sizeof(int32));
+ pq_endmessage(&msgbuf);
+
+ /*
+ * Hooray! Primary initialization is complete. Now, we need to set up
+ * our backend-local state to match the original backend.
+ */
+
+ /*
+ * Load libraries that were loaded by original backend. We want to do this
+ * before restoring GUCs, because the libraries might define custom
+ * variables.
+ */
+ libraryspace = shm_toc_lookup(toc, PARALLEL_KEY_LIBRARY);
+ Assert(libraryspace != NULL);
+ RestoreLibraryState(libraryspace);
+
+ /* Restore database connection. */
+ BackgroundWorkerInitializeConnectionByOid(fps->database_id,
+ fps->authenticated_user_id);
+
+ /* Restore GUC values from launching backend. */
+ gucspace = shm_toc_lookup(toc, PARALLEL_KEY_GUC);
+ Assert(gucspace != NULL);
+ StartTransactionCommand();
+ RestoreGUCState(gucspace);
+ CommitTransactionCommand();
+
+ /* Crank up a transaction state appropriate to a parallel worker. */
+ tstatespace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_STATE);
+ StartParallelWorkerTransaction(tstatespace);
+
+ /* Restore combo CID state. */
+ combocidspace = shm_toc_lookup(toc, PARALLEL_KEY_COMBO_CID);
+ Assert(combocidspace != NULL);
+ RestoreComboCIDState(combocidspace);
+
+ /* Restore transaction snapshot. */
+ tsnapspace = shm_toc_lookup(toc, PARALLEL_KEY_TRANSACTION_SNAPSHOT);
+ Assert(tsnapspace != NULL);
+ RestoreTransactionSnapshot(RestoreSnapshot(tsnapspace),
+ fps->parallel_master_pgproc);
+
+ /* Restore active snapshot. */
+ asnapspace = shm_toc_lookup(toc, PARALLEL_KEY_ACTIVE_SNAPSHOT);
+ Assert(asnapspace != NULL);
+ PushActiveSnapshot(RestoreSnapshot(asnapspace));
+
+ /* Restore user ID and security context. */
+ SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+
+ /*
+ * We've initialized all of our state now; nothing should change hereafter.
+ */
+ EnterParallelMode();
+
+ /*
+ * Time to do the real work: invoke the caller-supplied code.
+ *
+ * If you get a crash at this line, see the comments for
+ * ParallelExtensionTrampoline.
+ */
+ fps->entrypoint(seg, toc);
+
+ /* Must exit parallel mode to pop active snapshot. */
+ ExitParallelMode();
+
+ /* Must pop active snapshot so resowner.c doesn't complain. */
+ PopActiveSnapshot();
+
+ /* Shut down the parallel-worker transaction. */
+ EndParallelWorkerTransaction();
+
+ /* Report success. */
+ pq_putmessage('X', NULL, 0);
+}
+
+/*
+ * It's unsafe for the entrypoint invoked by ParallelWorkerMain to be a
+ * function living in a dynamically loaded module, because the module might
+ * not be loaded in every process, or might be loaded but not at the same
+ * address. To work around that problem, CreateParallelContextForExtension()
+ * arranges to call this function rather than calling the extension-provided
+ * function directly; and this function then looks up the real entrypoint and
+ * calls it.
+ */
+static void
+ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc)
+{
+ char *extensionstate;
+ char *library_name;
+ char *function_name;
+ parallel_worker_main_type entrypt;
+
+ extensionstate = shm_toc_lookup(toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE);
+ Assert(extensionstate != NULL);
+ library_name = extensionstate;
+ function_name = extensionstate + strlen(library_name) + 1;
+
+ entrypt = (parallel_worker_main_type)
+ load_external_function(library_name, function_name, true, NULL);
+ entrypt(seg, toc);
+}
+
+/*
+ * Give the user a hint that this is a message propagated from a parallel
+ * worker. Otherwise, it can sometimes be confusing to understand what
+ * actually happened.
+ */
+static void
+ParallelErrorContext(void *arg)
+{
+ errcontext("parallel worker, pid %d", * (int32 *) arg);
+}
+
+/*
+ * Update shared memory with the ending location of the last WAL record we
+ * wrote, if it's greater than the value already stored there.
+ */
+void
+ParallelWorkerReportLastRecEnd(XLogRecPtr last_xlog_end)
+{
+ FixedParallelState *fps = MyFixedParallelState;
+
+ Assert(fps != NULL);
+ SpinLockAcquire(&fps->mutex);
+ if (fps->last_xlog_end < last_xlog_end)
+ fps->last_xlog_end = last_xlog_end;
+ SpinLockRelease(&fps->mutex);
+}
diff --git a/src/backend/access/transam/varsup.c b/src/backend/access/transam/varsup.c
index 42ee57f..cf3e964 100644
--- a/src/backend/access/transam/varsup.c
+++ b/src/backend/access/transam/varsup.c
@@ -50,6 +50,13 @@ GetNewTransactionId(bool isSubXact)
TransactionId xid;
/*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new XIDs after that point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign TransactionIds during a parallel operation");
+
+ /*
* During bootstrap initialization, we return the special bootstrap
* transaction id.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index 1495bb4..ebc232a 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -22,6 +22,7 @@
#include "access/commit_ts.h"
#include "access/multixact.h"
+#include "access/parallel.h"
#include "access/subtrans.h"
#include "access/transam.h"
#include "access/twophase.h"
@@ -49,6 +50,7 @@
#include "storage/procarray.h"
#include "storage/sinvaladt.h"
#include "storage/smgr.h"
+#include "utils/builtins.h"
#include "utils/catcache.h"
#include "utils/combocid.h"
#include "utils/guc.h"
@@ -76,6 +78,33 @@ bool XactDeferrable;
int synchronous_commit = SYNCHRONOUS_COMMIT_ON;
/*
+ * When running as a parallel worker, we place only a single
+ * TransactionStateData on the parallel worker's state stack, and the XID
+ * reflected there will be that of the *innermost* currently-active
+ * subtransaction in the backend that initiated paralllelism. However,
+ * GetTopTransactionId() and TransactionIdIsCurrentTransactionId()
+ * need to return the same answers in the parallel worker as they would have
+ * in the user backend, so we need some additional bookkeeping.
+ *
+ * XactTopTransactionId stores the XID of our toplevel transaction, which
+ * will be the same as TopTransactionState.transactionId in an ordinary
+ * backend; but in a parallel backend, which does not have the entire
+ * transaction state, it will instead be copied from the backend that started
+ * the parallel operation.
+ *
+ * nParallelCurrentXids will be 0 and ParallelCurrentXids NULL in an ordinary
+ * backend, but in a parallel backend, nParallelCurrentXids will contain the
+ * number of XIDs that need to be considered current, and ParallelCurrentXids
+ * will contain the XIDs themselves. This includes all XIDs that were current
+ * or sub-committed in the parent at the time the parallel operation began.
+ * The XIDs are stored sorted in numerical order (not logical order) to make
+ * lookups as fast as possible.
+ */
+TransactionId XactTopTransactionId = InvalidTransactionId;
+int nParallelCurrentXids = 0;
+TransactionId *ParallelCurrentXids;
+
+/*
* MyXactAccessedTempRel is set when a temporary relation is accessed.
* We don't allow PREPARE TRANSACTION in that case. (This is global
* so that it can be set from heapam.c.)
@@ -111,6 +140,7 @@ typedef enum TBlockState
/* transaction block states */
TBLOCK_BEGIN, /* starting transaction block */
TBLOCK_INPROGRESS, /* live transaction */
+ TBLOCK_PARALLEL_INPROGRESS, /* live transaction inside parallel worker */
TBLOCK_END, /* COMMIT received */
TBLOCK_ABORT, /* failed xact, awaiting ROLLBACK */
TBLOCK_ABORT_END, /* failed xact, ROLLBACK received */
@@ -152,6 +182,7 @@ typedef struct TransactionStateData
bool prevXactReadOnly; /* entry-time xact r/o state */
bool startedInRecovery; /* did we start in recovery? */
bool didLogXid; /* has xid been included in WAL record? */
+ int parallelModeLevel; /* Enter/ExitParallelMode counter */
struct TransactionStateData *parent; /* back link to parent */
} TransactionStateData;
@@ -182,6 +213,7 @@ static TransactionStateData TopTransactionStateData = {
false, /* entry-time xact r/o state */
false, /* startedInRecovery */
false, /* didLogXid */
+ 0, /* parallelMode */
NULL /* link to parent state block */
};
@@ -351,9 +383,9 @@ IsAbortedTransactionBlockState(void)
TransactionId
GetTopTransactionId(void)
{
- if (!TransactionIdIsValid(TopTransactionStateData.transactionId))
+ if (!TransactionIdIsValid(XactTopTransactionId))
AssignTransactionId(&TopTransactionStateData);
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -366,7 +398,7 @@ GetTopTransactionId(void)
TransactionId
GetTopTransactionIdIfAny(void)
{
- return TopTransactionStateData.transactionId;
+ return XactTopTransactionId;
}
/*
@@ -460,6 +492,13 @@ AssignTransactionId(TransactionState s)
Assert(s->state == TRANS_INPROGRESS);
/*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new XIDs at this point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot assign XIDs during a parallel operation");
+
+ /*
* Ensure parent(s) have XIDs, so that a child always has an XID later
* than its parent. Musn't recurse here, or we might get a stack overflow
* if we're at the bottom of a huge stack of subtransactions none of which
@@ -511,6 +550,8 @@ AssignTransactionId(TransactionState s)
* the Xid as "running". See GetNewTransactionId.
*/
s->transactionId = GetNewTransactionId(isSubXact);
+ if (!isSubXact)
+ XactTopTransactionId = s->transactionId;
if (isSubXact)
SubTransSetParent(s->transactionId, s->parent->transactionId, false);
@@ -642,7 +683,16 @@ GetCurrentCommandId(bool used)
{
/* this is global to a transaction, not subtransaction-local */
if (used)
+ {
+ /*
+ * Forbid setting currentCommandIdUsed in parallel mode, because we
+ * have no provision for communicating this back to the master. We
+ * could relax this restriction when currentCommandIdUsed was already
+ * true at the start of the parallel operation.
+ */
+ Assert(CurrentTransactionState->parallelModeLevel == 0);
currentCommandIdUsed = true;
+ }
return currentCommandId;
}
@@ -736,6 +786,36 @@ TransactionIdIsCurrentTransactionId(TransactionId xid)
return false;
/*
+ * In parallel workers, the XIDs we must consider as current are stored
+ * in ParallelCurrentXids rather than the transaction-state stack. Note
+ * that the XIDs in this array are sorted numerically rather than
+ * according to transactionIdPrecedes order.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ int low,
+ high;
+
+ low = 0;
+ high = nParallelCurrentXids - 1;
+ while (low <= high)
+ {
+ int middle;
+ TransactionId probe;
+
+ middle = low + (high - low) / 2;
+ probe = ParallelCurrentXids[middle];
+ if (probe == xid)
+ return true;
+ else if (probe < xid)
+ low = middle + 1;
+ else
+ high = middle - 1;
+ }
+ return false;
+ }
+
+ /*
* We will return true for the Xid of the current subtransaction, any of
* its subcommitted children, any of its parents, or any of their
* previously subcommitted children. However, a transaction being aborted
@@ -789,6 +869,48 @@ TransactionStartedDuringRecovery(void)
}
/*
+ * EnterParallelMode
+ */
+void
+EnterParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel >= 0);
+
+ ++s->parallelModeLevel;
+}
+
+/*
+ * ExitParallelMode
+ */
+void
+ExitParallelMode(void)
+{
+ TransactionState s = CurrentTransactionState;
+
+ Assert(s->parallelModeLevel > 0);
+ Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
+
+ --s->parallelModeLevel;
+}
+
+/*
+ * IsInParallelMode
+ *
+ * Are we in a parallel operation, as either the master or a worker? Check
+ * this to prohibit operations that change backend-local state expected to
+ * match across all workers. Mere caches usually don't require such a
+ * restriction. State modified in a strict push/pop fashion, such as the
+ * active snapshot stack, is often fine.
+ */
+bool
+IsInParallelMode(void)
+{
+ return CurrentTransactionState->parallelModeLevel != 0;
+}
+
+/*
* CommandCounterIncrement
*/
void
@@ -802,6 +924,14 @@ CommandCounterIncrement(void)
*/
if (currentCommandIdUsed)
{
+ /*
+ * Workers synchronize transaction state at the beginning of each
+ * parallel operation, so we can't account for new commands after that
+ * point.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot start commands during a parallel operation");
+
currentCommandId += 1;
if (currentCommandId == InvalidCommandId)
{
@@ -1640,6 +1770,8 @@ StartTransaction(void)
s = &TopTransactionStateData;
CurrentTransactionState = s;
+ Assert(XactTopTransactionId == InvalidTransactionId);
+
/*
* check the current transaction state
*/
@@ -1769,6 +1901,9 @@ CommitTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool is_parallel_worker;
+
+ is_parallel_worker = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
ShowTransactionState("CommitTransaction");
@@ -1802,7 +1937,8 @@ CommitTransaction(void)
break;
}
- CallXactCallbacks(XACT_EVENT_PRE_COMMIT);
+ CallXactCallbacks(is_parallel_worker ? XACT_EVENT_PARALLEL_PRE_COMMIT
+ : XACT_EVENT_PRE_COMMIT);
/*
* The remaining actions cannot call any user-defined code, so it's safe
@@ -1811,6 +1947,13 @@ CommitTransaction(void)
* the transaction-abort path.
*/
+ /* If we might have parallel workers, clean them up now. */
+ if (IsInParallelMode())
+ {
+ AtEOXact_Parallel(true);
+ s->parallelModeLevel = 0;
+ }
+
/* Shut down the deferred-trigger manager */
AfterTriggerEndXact(true);
@@ -1849,10 +1992,28 @@ CommitTransaction(void)
*/
s->state = TRANS_COMMIT;
- /*
- * Here is where we really truly commit.
- */
- latestXid = RecordTransactionCommit();
+ if (!is_parallel_worker)
+ {
+ /*
+ * We need to mark our XIDs as commited in pg_clog. This is where we
+ * durably commit.
+ */
+ latestXid = RecordTransactionCommit();
+ }
+ else
+ {
+ /*
+ * We must not mark our XID committed; the parallel master is
+ * responsible for that.
+ */
+ latestXid = InvalidTransactionId;
+
+ /*
+ * Make sure the master will know about any WAL we wrote before it
+ * commits.
+ */
+ ParallelWorkerReportLastRecEnd(XactLastRecEnd);
+ }
TRACE_POSTGRESQL_TRANSACTION_COMMIT(MyProc->lxid);
@@ -1879,7 +2040,8 @@ CommitTransaction(void)
* state.
*/
- CallXactCallbacks(XACT_EVENT_COMMIT);
+ CallXactCallbacks(is_parallel_worker ? XACT_EVENT_PARALLEL_COMMIT
+ : XACT_EVENT_COMMIT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -1927,7 +2089,7 @@ CommitTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, is_parallel_worker);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -1952,6 +2114,9 @@ CommitTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with commit processing, set current transaction state back to
* default
@@ -1975,6 +2140,8 @@ PrepareTransaction(void)
GlobalTransaction gxact;
TimestampTz prepared_at;
+ Assert(!IsInParallelMode());
+
ShowTransactionState("PrepareTransaction");
/*
@@ -2194,7 +2361,7 @@ PrepareTransaction(void)
AtEOXact_GUC(true, 1);
AtEOXact_SPI(true);
AtEOXact_on_commit_actions(true);
- AtEOXact_Namespace(true);
+ AtEOXact_Namespace(true, false);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2219,6 +2386,9 @@ PrepareTransaction(void)
s->nChildXids = 0;
s->maxChildXids = 0;
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
+
/*
* done with 1st phase commit processing, set current transaction state
* back to default
@@ -2237,6 +2407,7 @@ AbortTransaction(void)
{
TransactionState s = CurrentTransactionState;
TransactionId latestXid;
+ bool is_parallel_worker;
/* Prevent cancel/die interrupt while cleaning up */
HOLD_INTERRUPTS();
@@ -2285,6 +2456,7 @@ AbortTransaction(void)
/*
* check the current transaction state
*/
+ is_parallel_worker = (s->blockState == TBLOCK_PARALLEL_INPROGRESS);
if (s->state != TRANS_INPROGRESS && s->state != TRANS_PREPARE)
elog(WARNING, "AbortTransaction while in %s state",
TransStateAsString(s->state));
@@ -2308,6 +2480,13 @@ AbortTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOXact_Parallel(false);
+ s->parallelModeLevel = 0;
+ }
+
/*
* do abort processing
*/
@@ -2320,9 +2499,23 @@ AbortTransaction(void)
/*
* Advertise the fact that we aborted in pg_clog (assuming that we got as
- * far as assigning an XID to advertise).
+ * far as assigning an XID to advertise). But if we're inside a parallel
+ * worker, skip this; the user backend must be the one to write the abort
+ * record.
*/
- latestXid = RecordTransactionAbort(false);
+ if (!is_parallel_worker)
+ 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);
+ }
TRACE_POSTGRESQL_TRANSACTION_ABORT(MyProc->lxid);
@@ -2340,7 +2533,10 @@ AbortTransaction(void)
*/
if (TopTransactionResourceOwner != NULL)
{
- CallXactCallbacks(XACT_EVENT_ABORT);
+ if (is_parallel_worker)
+ CallXactCallbacks(XACT_EVENT_PARALLEL_ABORT);
+ else
+ CallXactCallbacks(XACT_EVENT_ABORT);
ResourceOwnerRelease(TopTransactionResourceOwner,
RESOURCE_RELEASE_BEFORE_LOCKS,
@@ -2361,7 +2557,7 @@ AbortTransaction(void)
AtEOXact_GUC(false, 1);
AtEOXact_SPI(false);
AtEOXact_on_commit_actions(false);
- AtEOXact_Namespace(false);
+ AtEOXact_Namespace(false, is_parallel_worker);
AtEOXact_SMgr();
AtEOXact_Files();
AtEOXact_ComboCid();
@@ -2413,6 +2609,10 @@ CleanupTransaction(void)
s->childXids = NULL;
s->nChildXids = 0;
s->maxChildXids = 0;
+ s->parallelModeLevel = 0;
+
+ XactTopTransactionId = InvalidTransactionId;
+ nParallelCurrentXids = 0;
/*
* done with abort processing, set current transaction state back to
@@ -2466,6 +2666,7 @@ StartTransactionCommand(void)
/* These cases are invalid. */
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -2501,11 +2702,13 @@ CommitTransactionCommand(void)
switch (s->blockState)
{
/*
- * This shouldn't happen, because it means the previous
+ * These shouldn't happen. TBLOCK_DEFAULT means the previous
* StartTransactionCommand didn't set the STARTED state
- * appropriately.
+ * appropriately, while TBLOCK_PARALLEL_INPROGRESS should be ended
+ * by EndParallelWorkerTranaction(), not this function.
*/
case TBLOCK_DEFAULT:
+ case TBLOCK_PARALLEL_INPROGRESS:
elog(FATAL, "CommitTransactionCommand: unexpected state %s",
BlockStateAsString(s->blockState));
break;
@@ -2787,6 +2990,7 @@ AbortCurrentTransaction(void)
* ABORT state. We will stay in ABORT until we get a ROLLBACK.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
AbortTransaction();
s->blockState = TBLOCK_ABORT;
/* CleanupTransaction happens when we exit TBLOCK_ABORT_END */
@@ -3176,6 +3380,7 @@ BeginTransactionBlock(void)
* Already a transaction block in progress.
*/
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_ABORT:
case TBLOCK_SUBABORT:
@@ -3353,6 +3558,16 @@ EndTransactionBlock(void)
result = true;
break;
+ /*
+ * The user issued a COMMIT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3446,6 +3661,16 @@ UserAbortTransactionBlock(void)
s->blockState = TBLOCK_ABORT_PENDING;
break;
+ /*
+ * The user issued an ABORT that somehow ran inside a parallel
+ * worker. We can't cope with that.
+ */
+ case TBLOCK_PARALLEL_INPROGRESS:
+ ereport(FATAL,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot abort during a parallel operation")));
+ break;
+
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
@@ -3475,6 +3700,18 @@ DefineSavepoint(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot define savepoints during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_INPROGRESS:
@@ -3495,6 +3732,7 @@ DefineSavepoint(char *name)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3529,6 +3767,18 @@ ReleaseSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3552,6 +3802,7 @@ ReleaseSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3629,6 +3880,18 @@ RollbackToSavepoint(List *options)
ListCell *cell;
char *name = NULL;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for transaction state change after that
+ * point. (Note that this check will certainly error out if s->blockState
+ * is TBLOCK_PARALLEL_INPROGRESS, so we can treat that as an invalid case
+ * below.)
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot rollback to savepoints during a parallel operation")));
+
switch (s->blockState)
{
/*
@@ -3653,6 +3916,7 @@ RollbackToSavepoint(List *options)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -3741,6 +4005,20 @@ BeginInternalSubTransaction(char *name)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for new subtransactions after that point.
+ * We might be able to make an exception for the type of subtransaction
+ * established by this function, which is typically used in contexts where
+ * we're going to release or roll back the subtransaction before proceeding
+ * further, so that no enduring change to the transaction state occurs.
+ * For now, however, we prohibit this case along with all the others.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot start subtransactions during a parallel operation")));
+
switch (s->blockState)
{
case TBLOCK_STARTED:
@@ -3763,6 +4041,7 @@ BeginInternalSubTransaction(char *name)
/* These cases are invalid. */
case TBLOCK_DEFAULT:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_SUBRELEASE:
case TBLOCK_SUBCOMMIT:
@@ -3795,6 +4074,18 @@ ReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Workers synchronize transaction state at the beginning of each parallel
+ * operation, so we can't account for commit of subtransactions after that
+ * point. This should not happen anyway. Code calling this would
+ * typically have called BeginInternalSubTransaction() first, failing
+ * there.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot commit subtransactions during a parallel operation")));
+
if (s->blockState != TBLOCK_SUBINPROGRESS)
elog(ERROR, "ReleaseCurrentSubTransaction: unexpected state %s",
BlockStateAsString(s->blockState));
@@ -3817,6 +4108,14 @@ RollbackAndReleaseCurrentSubTransaction(void)
{
TransactionState s = CurrentTransactionState;
+ /*
+ * Unlike ReleaseCurrentSubTransaction(), this is nominally permitted
+ * during parallel operations. That's because we may be in the master,
+ * recovering from an error thrown while we were in parallel mode. We
+ * won't reach here in a worker, because BeginInternalSubTransaction()
+ * will have failed.
+ */
+
switch (s->blockState)
{
/* Must be in a subtransaction */
@@ -3828,6 +4127,7 @@ RollbackAndReleaseCurrentSubTransaction(void)
case TBLOCK_DEFAULT:
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
case TBLOCK_END:
@@ -3903,6 +4203,7 @@ AbortOutOfAnyTransaction(void)
case TBLOCK_STARTED:
case TBLOCK_BEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_END:
case TBLOCK_ABORT_PENDING:
case TBLOCK_PREPARE:
@@ -3994,6 +4295,7 @@ TransactionBlockStatusCode(void)
case TBLOCK_BEGIN:
case TBLOCK_SUBBEGIN:
case TBLOCK_INPROGRESS:
+ case TBLOCK_PARALLEL_INPROGRESS:
case TBLOCK_SUBINPROGRESS:
case TBLOCK_END:
case TBLOCK_SUBRELEASE:
@@ -4097,6 +4399,13 @@ CommitSubTransaction(void)
CallSubXactCallbacks(SUBXACT_EVENT_PRE_COMMIT_SUB, s->subTransactionId,
s->parent->subTransactionId);
+ /* If in parallel mode, clean up workers and exit parallel mode. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(true, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/* Do the actual "commit", such as it is */
s->state = TRANS_COMMIT;
@@ -4250,6 +4559,13 @@ AbortSubTransaction(void)
*/
SetUserIdAndSecContext(s->prevUser, s->prevSecContext);
+ /* Exit from parallel mode, if necessary. */
+ if (IsInParallelMode())
+ {
+ AtEOSubXact_Parallel(false, s->subTransactionId);
+ s->parallelModeLevel = 0;
+ }
+
/*
* We can skip all this stuff if the subxact failed before creating a
* ResourceOwner...
@@ -4390,6 +4706,7 @@ PushTransaction(void)
s->blockState = TBLOCK_SUBBEGIN;
GetUserIdAndSecContext(&s->prevUser, &s->prevSecContext);
s->prevXactReadOnly = XactReadOnly;
+ s->parallelModeLevel = 0;
CurrentTransactionState = s;
@@ -4437,6 +4754,139 @@ PopTransaction(void)
}
/*
+ * EstimateTransactionStateSpace
+ * Estimate the amount of space that will be needed by
+ * SerializeTransactionState. It would be OK to overestimate slightly,
+ * but it's simple for us to work out the precise value, so we do.
+ */
+Size
+EstimateTransactionStateSpace(void)
+{
+ TransactionState s;
+ Size nxids = 5; /* iso level, deferrable, top & current XID, XID count */
+
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+
+ nxids = add_size(nxids, nParallelCurrentXids);
+ return mul_size(nxids, sizeof(TransactionId));
+}
+
+/*
+ * SerializeTransactionState
+ * Write out relevant details of our transaction state that will be
+ * needed by a parallel worker.
+ *
+ * We need to save and restore XactDeferrable, XactIsoLevel, and the XIDs
+ * associated with this transaction. The first eight bytes of the result
+ * contain XactDeferrable and XactIsoLevel; the next eight bytes contain the
+ * XID of the top-level transaction and the XID of the current transaction
+ * (or, in each case, InvalidTransactionId if none). After that, the next 4
+ * bytes contain a count of how many additional XIDs follow; this is followed
+ * by all of those XIDs one after another. We emit the XIDs in sorted order
+ * for the convenience of the receiving process.
+ */
+void
+SerializeTransactionState(Size maxsize, char *start_address)
+{
+ TransactionState s;
+ Size nxids = 0;
+ Size i = 0;
+ TransactionId *workspace;
+ TransactionId *result = (TransactionId *) start_address;
+
+ Assert(maxsize >= 5 * sizeof(TransactionId));
+ result[0] = (TransactionId) XactIsoLevel;
+ result[1] = (TransactionId) XactDeferrable;
+ result[2] = XactTopTransactionId;
+ result[3] = CurrentTransactionState->transactionId;
+
+ /*
+ * If we're running in a parallel worker and launching a parallel worker
+ * of our own, we can just pass along the information that was passed to
+ * us.
+ */
+ if (nParallelCurrentXids > 0)
+ {
+ Assert(maxsize > (nParallelCurrentXids + 4) * sizeof(TransactionId));
+ result[4] = nParallelCurrentXids;
+ memcpy(&result[5], ParallelCurrentXids,
+ nParallelCurrentXids * sizeof(TransactionId));
+ return;
+ }
+
+ /*
+ * OK, we need to generate a sorted list of XIDs that our workers
+ * should view as current. First, figure out how many there are.
+ */
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ nxids = add_size(nxids, 1);
+ nxids = add_size(nxids, s->nChildXids);
+ }
+ Assert(nxids * sizeof(TransactionId) < maxsize);
+
+ /* Copy them to our scratch space. */
+ workspace = palloc(nxids * sizeof(TransactionId));
+ for (s = CurrentTransactionState; s != NULL; s = s->parent)
+ {
+ if (TransactionIdIsValid(s->transactionId))
+ workspace[i++] = s->transactionId;
+ memcpy(&workspace[i], s->childXids,
+ s->nChildXids * sizeof(TransactionId));
+ i += s->nChildXids;
+ }
+ Assert(i == nxids);
+
+ /* Sort them. */
+ qsort(workspace, nxids, sizeof(TransactionId), xidComparator);
+
+ /* Copy data into output area. */
+ result[4] = (TransactionId) nxids;
+ memcpy(&result[5], workspace, nxids * sizeof(TransactionId));
+}
+
+/*
+ * StartParallelWorkerTransaction
+ * Start a parallel worker transaction, restoring the relevant
+ * transaction state serialized by SerializeTransactionState.
+ */
+void
+StartParallelWorkerTransaction(char *tstatespace)
+{
+ TransactionId *tstate = (TransactionId *) tstatespace;
+
+ Assert(CurrentTransactionState->blockState == TBLOCK_DEFAULT);
+ StartTransaction();
+
+ XactIsoLevel = (int) tstate[0];
+ XactDeferrable = (bool) tstate[1];
+ XactTopTransactionId = tstate[2];
+ CurrentTransactionState->transactionId = tstate[3];
+ nParallelCurrentXids = (int) tstate[4];
+ ParallelCurrentXids = &tstate[5];
+
+ CurrentTransactionState->blockState = TBLOCK_PARALLEL_INPROGRESS;
+}
+
+/*
+ * EndParallelWorkerTransaction
+ * End a parallel worker transaction.
+ */
+void
+EndParallelWorkerTransaction(void)
+{
+ Assert(CurrentTransactionState->blockState == TBLOCK_PARALLEL_INPROGRESS);
+ CommitTransaction();
+ CurrentTransactionState->blockState = TBLOCK_DEFAULT;
+}
+
+/*
* ShowTransactionState
* Debug support
*/
@@ -4506,6 +4956,8 @@ BlockStateAsString(TBlockState blockState)
return "BEGIN";
case TBLOCK_INPROGRESS:
return "INPROGRESS";
+ case TBLOCK_PARALLEL_INPROGRESS:
+ return "PARALLEL_INPROGRESS";
case TBLOCK_END:
return "END";
case TBLOCK_ABORT:
diff --git a/src/backend/access/transam/xlog.c b/src/backend/access/transam/xlog.c
index 2580996..3105d10 100644
--- a/src/backend/access/transam/xlog.c
+++ b/src/backend/access/transam/xlog.c
@@ -291,6 +291,14 @@ static TimeLineID curFileTLI;
* end+1 of the last record, and is reset when we end a top-level transaction,
* or start a new one; so it can be used to tell if the current transaction has
* created any XLOG records.
+ *
+ * While in parallel mode, this may not be fully up to date. When committing,
+ * a transaction can assume this covers all xlog records written either by the
+ * user backend or by any parallel worker which was present at any point during
+ * the transaction. But when aborting, or when still in parallel mode, other
+ * parallel backends may have written WAL records at later LSNs than the value
+ * stored here. The parallel leader advances its own copy, when necessary,
+ * in WaitForParallelWorkersToFinish.
*/
static XLogRecPtr ProcLastRecPtr = InvalidXLogRecPtr;
diff --git a/src/backend/catalog/namespace.c b/src/backend/catalog/namespace.c
index 1af977c..2f6d697 100644
--- a/src/backend/catalog/namespace.c
+++ b/src/backend/catalog/namespace.c
@@ -20,6 +20,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/parallel.h"
#include "access/xact.h"
#include "access/xlog.h"
#include "catalog/dependency.h"
@@ -3646,6 +3647,12 @@ InitTempTableNamespace(void)
(errcode(ERRCODE_READ_ONLY_SQL_TRANSACTION),
errmsg("cannot create temporary tables during recovery")));
+ /* Parallel workers can't create temporary tables, either. */
+ if (IsParallelWorker())
+ ereport(ERROR,
+ (errcode(ERRCODE_READ_ONLY_SQL_TRANSACTION),
+ errmsg("cannot create temporary tables in parallel mode")));
+
snprintf(namespaceName, sizeof(namespaceName), "pg_temp_%d", MyBackendId);
namespaceId = get_namespace_oid(namespaceName, true);
@@ -3709,7 +3716,7 @@ InitTempTableNamespace(void)
* End-of-transaction cleanup for namespaces.
*/
void
-AtEOXact_Namespace(bool isCommit)
+AtEOXact_Namespace(bool isCommit, bool parallel)
{
/*
* If we abort the transaction in which a temp namespace was selected,
@@ -3719,7 +3726,7 @@ AtEOXact_Namespace(bool isCommit)
* at backend shutdown. (We only want to register the callback once per
* session, so this is a good place to do it.)
*/
- if (myTempNamespaceSubID != InvalidSubTransactionId)
+ if (myTempNamespaceSubID != InvalidSubTransactionId && !parallel)
{
if (isCommit)
before_shmem_exit(RemoveTempRelationsCallback, 0);
diff --git a/src/backend/commands/copy.c b/src/backend/commands/copy.c
index 92ff632..0d3721a 100644
--- a/src/backend/commands/copy.c
+++ b/src/backend/commands/copy.c
@@ -924,9 +924,10 @@ DoCopy(const CopyStmt *stmt, const char *queryString, uint64 *processed)
{
Assert(rel);
- /* check read-only transaction */
+ /* check read-only transaction and parallel mode */
if (XactReadOnly && !rel->rd_islocaltemp)
PreventCommandIfReadOnly("COPY FROM");
+ PreventCommandIfParallelMode("COPY FROM");
cstate = BeginCopyFrom(rel, stmt->filename, stmt->is_program,
stmt->attlist, stmt->options);
diff --git a/src/backend/commands/sequence.c b/src/backend/commands/sequence.c
index 6d316d6..80f5553 100644
--- a/src/backend/commands/sequence.c
+++ b/src/backend/commands/sequence.c
@@ -565,6 +565,13 @@ nextval_internal(Oid relid)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("nextval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("nextval()");
+
if (elm->last != elm->cached) /* some numbers were cached */
{
Assert(elm->last_valid);
@@ -862,6 +869,13 @@ do_setval(Oid relid, int64 next, bool iscalled)
if (!seqrel->rd_islocaltemp)
PreventCommandIfReadOnly("setval()");
+ /*
+ * Forbid this during parallel operation because, to make it work,
+ * the cooperating backends would need to share the backend-local cached
+ * sequence information. Currently, we don't support that.
+ */
+ PreventCommandIfParallelMode("setval()");
+
/* lock page' buffer and read tuple */
seq = read_seq_tuple(elm, seqrel, &buf, &seqtuple);
diff --git a/src/backend/executor/execMain.c b/src/backend/executor/execMain.c
index df4da3f..26793ee 100644
--- a/src/backend/executor/execMain.c
+++ b/src/backend/executor/execMain.c
@@ -147,8 +147,20 @@ standard_ExecutorStart(QueryDesc *queryDesc, int eflags)
/*
* If the transaction is read-only, we need to check if any writes are
* planned to non-temporary tables. EXPLAIN is considered read-only.
+ *
+ * Don't allow writes in parallel mode. Supporting UPDATE and DELETE would
+ * require (a) storing the combocid hash in shared memory, rather than
+ * synchronizing it just once at the start of parallelism, and (b) an
+ * alternative to heap_update()'s reliance on xmax for mutual exclusion.
+ * INSERT may have no such troubles, but we forbid it to simplify the
+ * checks.
+ *
+ * We have lower-level defenses in CommandCounterIncrement and elsewhere
+ * against performing unsafe operations in parallel mode, but this gives
+ * a more user-friendly error message.
*/
- if (XactReadOnly && !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
+ if ((XactReadOnly || IsInParallelMode()) &&
+ !(eflags & EXEC_FLAG_EXPLAIN_ONLY))
ExecCheckXactReadOnly(queryDesc->plannedstmt);
/*
@@ -691,18 +703,23 @@ ExecCheckRTEPerms(RangeTblEntry *rte)
}
/*
- * Check that the query does not imply any writes to non-temp tables.
+ * Check that the query does not imply any writes to non-temp tables;
+ * unless we're in parallel mode, in which case don't even allow writes
+ * to temp tables.
*
* Note: in a Hot Standby slave this would need to reject writes to temp
- * tables as well; but an HS slave can't have created any temp tables
- * in the first place, so no need to check that.
+ * tables just as we do in parallel mode; but an HS slave can't have created
+ * any temp tables in the first place, so no need to check that.
*/
static void
ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
{
ListCell *l;
- /* Fail if write permissions are requested on any non-temp table */
+ /*
+ * Fail if write permissions are requested in parallel mode for
+ * table (temp or non-temp), otherwise fail for any non-temp table.
+ */
foreach(l, plannedstmt->rtable)
{
RangeTblEntry *rte = (RangeTblEntry *) lfirst(l);
@@ -718,6 +735,9 @@ ExecCheckXactReadOnly(PlannedStmt *plannedstmt)
PreventCommandIfReadOnly(CreateCommandTag((Node *) plannedstmt));
}
+
+ if (plannedstmt->commandType != CMD_SELECT || plannedstmt->hasModifyingCTE)
+ PreventCommandIfParallelMode(CreateCommandTag((Node *) plannedstmt));
}
diff --git a/src/backend/executor/functions.c b/src/backend/executor/functions.c
index 6c3eff7..ce49c47 100644
--- a/src/backend/executor/functions.c
+++ b/src/backend/executor/functions.c
@@ -513,6 +513,9 @@ init_execution_state(List *queryTree_list,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/* OK, build the execution_state for this query */
newes = (execution_state *) palloc(sizeof(execution_state));
if (preves)
diff --git a/src/backend/executor/spi.c b/src/backend/executor/spi.c
index b3c0502..557d153 100644
--- a/src/backend/executor/spi.c
+++ b/src/backend/executor/spi.c
@@ -23,6 +23,7 @@
#include "commands/trigger.h"
#include "executor/executor.h"
#include "executor/spi_priv.h"
+#include "miscadmin.h"
#include "tcop/pquery.h"
#include "tcop/utility.h"
#include "utils/builtins.h"
@@ -1322,13 +1323,14 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
}
/*
- * If told to be read-only, we'd better check for read-only queries. This
- * can't be done earlier because we need to look at the finished, planned
- * queries. (In particular, we don't want to do it between GetCachedPlan
- * and PortalDefineQuery, because throwing an error between those steps
- * would result in leaking our plancache refcount.)
+ * If told to be read-only, or in parallel mode, verify that this query
+ * is in fact read-only. This can't be done earlier because we need to
+ * look at the finished, planned queries. (In particular, we don't want
+ * to do it between GetCachedPlan and PortalDefineQuery, because throwing
+ * an error between those steps would result in leaking our plancache
+ * refcount.)
*/
- if (read_only)
+ if (read_only || IsInParallelMode())
{
ListCell *lc;
@@ -1337,11 +1339,16 @@ SPI_cursor_open_internal(const char *name, SPIPlanPtr plan,
Node *pstmt = (Node *) lfirst(lc);
if (!CommandIsReadOnly(pstmt))
- ereport(ERROR,
- (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
- /* translator: %s is a SQL statement name */
- errmsg("%s is not allowed in a non-volatile function",
- CreateCommandTag(pstmt))));
+ {
+ if (read_only)
+ ereport(ERROR,
+ (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
+ /* translator: %s is a SQL statement name */
+ errmsg("%s is not allowed in a non-volatile function",
+ CreateCommandTag(pstmt))));
+ else
+ PreventCommandIfParallelMode(CreateCommandTag(pstmt));
+ }
}
}
@@ -2129,6 +2136,9 @@ _SPI_execute_plan(SPIPlanPtr plan, ParamListInfo paramLI,
errmsg("%s is not allowed in a non-volatile function",
CreateCommandTag(stmt))));
+ if (IsInParallelMode() && !CommandIsReadOnly(stmt))
+ PreventCommandIfParallelMode(CreateCommandTag(stmt));
+
/*
* If not read-only mode, advance the command counter before each
* command and update the snapshot.
diff --git a/src/backend/libpq/pqmq.c b/src/backend/libpq/pqmq.c
index 307fb60..f12f2d5 100644
--- a/src/backend/libpq/pqmq.c
+++ b/src/backend/libpq/pqmq.c
@@ -16,12 +16,15 @@
#include "libpq/libpq.h"
#include "libpq/pqformat.h"
#include "libpq/pqmq.h"
+#include "miscadmin.h"
#include "tcop/tcopprot.h"
#include "utils/builtins.h"
static shm_mq *pq_mq;
static shm_mq_handle *pq_mq_handle;
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 void mq_comm_reset(void);
static int mq_flush(void);
@@ -57,6 +60,18 @@ pq_redirect_to_shm_mq(shm_mq *mq, shm_mq_handle *mqh)
FrontendProtocol = PG_PROTOCOL_LATEST;
}
+/*
+ * Arrange to SendProcSignal() to the parallel master each time we transmit
+ * message data via the shm_mq.
+ */
+void
+pq_set_parallel_master(pid_t pid, BackendId backend_id)
+{
+ Assert(PqCommMethods == &PqCommMqMethods);
+ pq_mq_parallel_master_pid = pid;
+ pq_mq_parallel_master_backend_id = backend_id;
+}
+
static void
mq_comm_reset(void)
{
@@ -120,7 +135,23 @@ mq_putmessage(char msgtype, const char *s, size_t len)
iov[1].len = len;
Assert(pq_mq_handle != NULL);
- result = shm_mq_sendv(pq_mq_handle, iov, 2, false);
+
+ for (;;)
+ {
+ result = shm_mq_sendv(pq_mq_handle, iov, 2, true);
+
+ if (pq_mq_parallel_master_pid != 0)
+ SendProcSignal(pq_mq_parallel_master_pid,
+ PROCSIG_PARALLEL_MESSAGE,
+ pq_mq_parallel_master_backend_id);
+
+ if (result != SHM_MQ_WOULD_BLOCK)
+ break;
+
+ WaitLatch(&MyProc->procLatch, WL_LATCH_SET, 0);
+ CHECK_FOR_INTERRUPTS();
+ ResetLatch(&MyProc->procLatch);
+ }
pq_mq_busy = false;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index d493941..3777333 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -996,6 +996,56 @@ WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *handle, pid_t *pidp)
}
/*
+ * Wait for a background worker to stop.
+ *
+ * If the worker hasn't yet started, or is running, we wait for it to stop
+ * and then return BGWH_STOPPED. However, if the postmaster has died, we give
+ * up and return BGWH_POSTMASTER_DIED, because it's the postmaster that
+ * notifies us when a worker's state changes.
+ */
+BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *handle)
+{
+ BgwHandleStatus status;
+ int rc;
+ bool save_set_latch_on_sigusr1;
+
+ save_set_latch_on_sigusr1 = set_latch_on_sigusr1;
+ set_latch_on_sigusr1 = true;
+
+ PG_TRY();
+ {
+ for (;;)
+ {
+ pid_t pid;
+
+ CHECK_FOR_INTERRUPTS();
+
+ status = GetBackgroundWorkerPid(handle, &pid);
+ if (status == BGWH_STOPPED)
+ return status;
+
+ rc = WaitLatch(&MyProc->procLatch,
+ WL_LATCH_SET | WL_POSTMASTER_DEATH, 0);
+
+ if (rc & WL_POSTMASTER_DEATH)
+ return BGWH_POSTMASTER_DIED;
+
+ ResetLatch(&MyProc->procLatch);
+ }
+ }
+ PG_CATCH();
+ {
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ PG_RE_THROW();
+ }
+ PG_END_TRY();
+
+ set_latch_on_sigusr1 = save_set_latch_on_sigusr1;
+ return status;
+}
+
+/*
* Instruct the postmaster to terminate a background worker.
*
* Note that it's safe to do this without regard to whether the worker is
diff --git a/src/backend/storage/ipc/procarray.c b/src/backend/storage/ipc/procarray.c
index 8eaec0c..68cc6ed 100644
--- a/src/backend/storage/ipc/procarray.c
+++ b/src/backend/storage/ipc/procarray.c
@@ -1683,6 +1683,50 @@ ProcArrayInstallImportedXmin(TransactionId xmin, TransactionId sourcexid)
}
/*
+ * ProcArrayInstallRestoredXmin -- install restored xmin into MyPgXact->xmin
+ *
+ * This is like ProcArrayInstallImportedXmin, but we have a pointer to the
+ * PGPROC of the transaction from which we imported the snapshot, rather than
+ * an XID.
+ *
+ * Returns TRUE if successful, FALSE if source xact is no longer running.
+ */
+bool
+ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc)
+{
+ bool result = false;
+ TransactionId xid;
+ volatile PGXACT *pgxact;
+
+ Assert(TransactionIdIsNormal(xmin));
+ Assert(proc != NULL);
+
+ /* Get lock so source xact can't end while we're doing this */
+ LWLockAcquire(ProcArrayLock, LW_SHARED);
+
+ pgxact = &allPgXact[proc->pgprocno];
+
+ /*
+ * Be certain that the referenced PGPROC has an advertised xmin which
+ * is no later than the one we're installing, so that the system-wide
+ * xmin can't go backwards. Also, make sure it's running in the same
+ * database, so that the per-database xmin cannot go backwards.
+ */
+ xid = pgxact->xmin; /* fetch just once */
+ if (proc->databaseId == MyDatabaseId &&
+ TransactionIdIsNormal(xid) &&
+ TransactionIdPrecedesOrEquals(xid, xmin))
+ {
+ MyPgXact->xmin = TransactionXmin = xmin;
+ result = true;
+ }
+
+ LWLockRelease(ProcArrayLock);
+
+ return result;
+}
+
+/*
* GetRunningTransactionData -- returns information about running transactions.
*
* Similar to GetSnapshotData but returns more information. We include
diff --git a/src/backend/storage/ipc/procsignal.c b/src/backend/storage/ipc/procsignal.c
index 48573be..0abde43 100644
--- a/src/backend/storage/ipc/procsignal.c
+++ b/src/backend/storage/ipc/procsignal.c
@@ -17,6 +17,7 @@
#include <signal.h>
#include <unistd.h>
+#include "access/parallel.h"
#include "commands/async.h"
#include "miscadmin.h"
#include "storage/latch.h"
@@ -274,6 +275,9 @@ procsignal_sigusr1_handler(SIGNAL_ARGS)
if (CheckProcSignal(PROCSIG_NOTIFY_INTERRUPT))
HandleNotifyInterrupt();
+ if (CheckProcSignal(PROCSIG_PARALLEL_MESSAGE))
+ HandleParallelMessageInterrupt();
+
if (CheckProcSignal(PROCSIG_RECOVERY_CONFLICT_DATABASE))
RecoveryConflictInterrupt(PROCSIG_RECOVERY_CONFLICT_DATABASE);
diff --git a/src/backend/storage/lmgr/predicate.c b/src/backend/storage/lmgr/predicate.c
index b81ebeb..01e03f0 100644
--- a/src/backend/storage/lmgr/predicate.c
+++ b/src/backend/storage/lmgr/predicate.c
@@ -1653,6 +1653,14 @@ GetSerializableTransactionSnapshotInt(Snapshot snapshot,
Assert(!RecoveryInProgress());
+ /*
+ * Since all parts of a serializable transaction must use the same
+ * snapshot, it is too late to establish one after a parallel operation
+ * has begun.
+ */
+ if (IsInParallelMode())
+ elog(ERROR, "cannot establish serializable snapshot during a parallel operation");
+
proc = MyProc;
Assert(proc != NULL);
GET_VXID_FROM_PGPROC(vxid, *proc);
diff --git a/src/backend/tcop/postgres.c b/src/backend/tcop/postgres.c
index 33720e8..ea2a432 100644
--- a/src/backend/tcop/postgres.c
+++ b/src/backend/tcop/postgres.c
@@ -36,6 +36,7 @@
#include "rusagestub.h"
#endif
+#include "access/parallel.h"
#include "access/printtup.h"
#include "access/xact.h"
#include "catalog/pg_type.h"
@@ -2988,7 +2989,8 @@ ProcessInterrupts(void)
}
}
- /* If we get here, do nothing (probably, QueryCancelPending was reset) */
+ if (ParallelMessagePending)
+ HandleParallelMessages();
}
diff --git a/src/backend/tcop/utility.c b/src/backend/tcop/utility.c
index 31e9d4c..59f09dc 100644
--- a/src/backend/tcop/utility.c
+++ b/src/backend/tcop/utility.c
@@ -128,14 +128,15 @@ CommandIsReadOnly(Node *parsetree)
static void
check_xact_readonly(Node *parsetree)
{
- if (!XactReadOnly)
+ /* Only perform the check if we have a reason to do so. */
+ if (!XactReadOnly && !IsInParallelMode())
return;
/*
* Note: Commands that need to do more complicated checking are handled
* elsewhere, in particular COPY and plannable statements do their own
- * checking. However they should all call PreventCommandIfReadOnly to
- * actually throw the error.
+ * checking. However they should all call PreventCommandIfReadOnly
+ * or PreventCommandIfParallelMode to actually throw the error.
*/
switch (nodeTag(parsetree))
@@ -208,6 +209,7 @@ check_xact_readonly(Node *parsetree)
case T_ImportForeignSchemaStmt:
case T_SecLabelStmt:
PreventCommandIfReadOnly(CreateCommandTag(parsetree));
+ PreventCommandIfParallelMode(CreateCommandTag(parsetree));
break;
default:
/* do nothing */
@@ -233,6 +235,24 @@ PreventCommandIfReadOnly(const char *cmdname)
}
/*
+ * PreventCommandIfParallelMode: throw error if current (sub)transaction is
+ * in parallel mode.
+ *
+ * This is useful mainly to ensure consistency of the error message wording;
+ * most callers have checked IsInParallelMode() for themselves.
+ */
+void
+PreventCommandIfParallelMode(const char *cmdname)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ /* translator: %s is name of a SQL command, eg CREATE */
+ errmsg("cannot execute %s during a parallel operation",
+ cmdname)));
+}
+
+/*
* PreventCommandDuringRecovery: throw error if RecoveryInProgress
*
* The majority of operations that are unsafe in a Hot Standby slave
@@ -618,6 +638,7 @@ standard_ProcessUtility(Node *parsetree,
case T_ClusterStmt:
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("CLUSTER");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
cluster((ClusterStmt *) parsetree, isTopLevel);
break;
@@ -628,6 +649,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery((stmt->options & VACOPT_VACUUM) ?
"VACUUM" : "ANALYZE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
ExecVacuum(stmt, isTopLevel);
}
break;
@@ -704,6 +726,7 @@ standard_ProcessUtility(Node *parsetree,
* outside a transaction block is presumed to be user error.
*/
RequireTransactionChain(isTopLevel, "LOCK TABLE");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
LockTableCommand((LockStmt *) parsetree);
break;
@@ -735,6 +758,7 @@ standard_ProcessUtility(Node *parsetree,
/* we choose to allow this during "read only" transactions */
PreventCommandDuringRecovery("REINDEX");
+ /* forbidden in parallel mode due to CommandIsReadOnly */
switch (stmt->kind)
{
case REINDEX_OBJECT_INDEX:
diff --git a/src/backend/utils/adt/lockfuncs.c b/src/backend/utils/adt/lockfuncs.c
index a1967b69..491824d 100644
--- a/src/backend/utils/adt/lockfuncs.c
+++ b/src/backend/utils/adt/lockfuncs.c
@@ -13,6 +13,7 @@
#include "postgres.h"
#include "access/htup_details.h"
+#include "access/xact.h"
#include "catalog/pg_type.h"
#include "funcapi.h"
#include "miscadmin.h"
@@ -411,6 +412,15 @@ pg_lock_status(PG_FUNCTION_ARGS)
#define SET_LOCKTAG_INT32(tag, key1, key2) \
SET_LOCKTAG_ADVISORY(tag, MyDatabaseId, key1, key2, 2)
+static void
+PreventAdvisoryLocksInParallelMode(void)
+{
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot use advisory locks during a parallel operation")));
+}
+
/*
* pg_advisory_lock(int8) - acquire exclusive lock on an int8 key
*/
@@ -420,6 +430,7 @@ pg_advisory_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -437,6 +448,7 @@ pg_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -453,6 +465,7 @@ pg_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -470,6 +483,7 @@ pg_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
int64 key = PG_GETARG_INT64(0);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -489,6 +503,7 @@ pg_try_advisory_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -509,6 +524,7 @@ pg_try_advisory_xact_lock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -528,6 +544,7 @@ pg_try_advisory_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -548,6 +565,7 @@ pg_try_advisory_xact_lock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -567,6 +585,7 @@ pg_advisory_unlock_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -586,6 +605,7 @@ pg_advisory_unlock_shared_int8(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT64(tag, key);
res = LockRelease(&tag, ShareLock, true);
@@ -603,6 +623,7 @@ pg_advisory_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, true, false);
@@ -621,6 +642,7 @@ pg_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ExclusiveLock, false, false);
@@ -638,6 +660,7 @@ pg_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, true, false);
@@ -656,6 +679,7 @@ pg_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
int32 key2 = PG_GETARG_INT32(1);
LOCKTAG tag;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
(void) LockAcquire(&tag, ShareLock, false, false);
@@ -676,6 +700,7 @@ pg_try_advisory_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, true, true);
@@ -697,6 +722,7 @@ pg_try_advisory_xact_lock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ExclusiveLock, false, true);
@@ -717,6 +743,7 @@ pg_try_advisory_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, true, true);
@@ -738,6 +765,7 @@ pg_try_advisory_xact_lock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
LockAcquireResult res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockAcquire(&tag, ShareLock, false, true);
@@ -758,6 +786,7 @@ pg_advisory_unlock_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ExclusiveLock, true);
@@ -778,6 +807,7 @@ pg_advisory_unlock_shared_int4(PG_FUNCTION_ARGS)
LOCKTAG tag;
bool res;
+ PreventAdvisoryLocksInParallelMode();
SET_LOCKTAG_INT32(tag, key1, key2);
res = LockRelease(&tag, ShareLock, true);
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 7476a26..46bc1f2 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -23,6 +23,7 @@
#endif
#include "lib/stringinfo.h"
#include "miscadmin.h"
+#include "storage/shmem.h"
#include "utils/dynamic_loader.h"
#include "utils/hsearch.h"
@@ -692,3 +693,56 @@ find_rendezvous_variable(const char *varName)
return &hentry->varValue;
}
+
+/*
+ * Estimate the amount of space needed to serialize the list of libraries
+ * we have loaded.
+ */
+Size
+EstimateLibraryStateSpace(void)
+{
+ DynamicFileList *file_scanner;
+ Size size = 1;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ size = add_size(size, strlen(file_scanner->filename) + 1);
+
+ return size;
+}
+
+/*
+ * Serialize the list of libraries we have loaded to a chunk of memory.
+ */
+void
+SerializeLibraryState(Size maxsize, char *start_address)
+{
+ DynamicFileList *file_scanner;
+
+ for (file_scanner = file_list;
+ file_scanner != NULL;
+ file_scanner = file_scanner->next)
+ {
+ Size len;
+
+ len = strlcpy(start_address, file_scanner->filename, maxsize) + 1;
+ Assert(len < maxsize);
+ maxsize -= len;
+ start_address += len;
+ }
+ start_address[0] = '\0';
+}
+
+/*
+ * Load every library the serializing backend had loaded.
+ */
+void
+RestoreLibraryState(char *start_address)
+{
+ while (*start_address != '\0')
+ {
+ internal_load_library(start_address);
+ start_address += strlen(start_address) + 1;
+ }
+}
diff --git a/src/backend/utils/misc/guc.c b/src/backend/utils/misc/guc.c
index f43aff2..8727ee3 100644
--- a/src/backend/utils/misc/guc.c
+++ b/src/backend/utils/misc/guc.c
@@ -5665,6 +5665,20 @@ set_config_option(const char *name, const char *value,
elevel = ERROR;
}
+ /*
+ * GUC_ACTION_SAVE changes are acceptable during a parallel operation,
+ * because the current worker will also pop the change. We're probably
+ * dealing with a function having a proconfig entry. Only the function's
+ * body should observe the change, and peer workers do not share in the
+ * execution of a function call started by this worker.
+ *
+ * Other changes might need to affect other workers, so forbid them.
+ */
+ if (IsInParallelMode() && changeVal && action != GUC_ACTION_SAVE)
+ ereport(elevel,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
record = find_option(name, true, elevel);
if (record == NULL)
{
@@ -6969,6 +6983,15 @@ ExecSetVariableStmt(VariableSetStmt *stmt, bool isTopLevel)
{
GucAction action = stmt->is_local ? GUC_ACTION_LOCAL : GUC_ACTION_SET;
+ /*
+ * Workers synchronize these parameters at the start of the parallel
+ * operation; then, we block SET during the operation.
+ */
+ if (IsInParallelMode())
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot set parameters during a parallel operation")));
+
switch (stmt->kind)
{
case VAR_SET_VALUE:
diff --git a/src/backend/utils/time/combocid.c b/src/backend/utils/time/combocid.c
index bfd7d0a..cc5409b 100644
--- a/src/backend/utils/time/combocid.c
+++ b/src/backend/utils/time/combocid.c
@@ -44,6 +44,7 @@
#include "miscadmin.h"
#include "access/htup_details.h"
#include "access/xact.h"
+#include "storage/shmem.h"
#include "utils/combocid.h"
#include "utils/hsearch.h"
#include "utils/memutils.h"
@@ -286,3 +287,76 @@ GetRealCmax(CommandId combocid)
Assert(combocid < usedComboCids);
return comboCids[combocid].cmax;
}
+
+/*
+ * Estimate the amount of space required to serialize the current ComboCID
+ * state.
+ */
+Size
+EstimateComboCIDStateSpace(void)
+{
+ Size size;
+
+ /* Add space required for saving usedComboCids */
+ size = sizeof(int);
+
+ /* Add space required for saving the combocids key */
+ size = add_size(size, mul_size(sizeof(ComboCidKeyData), usedComboCids));
+
+ return size;
+}
+
+/*
+ * Serialize the ComboCID state into the memory, beginning at start_address.
+ * maxsize should be at least as large as the value returned by
+ * EstimateComboCIDStateSpace.
+ */
+void
+SerializeComboCIDState(Size maxsize, char *start_address)
+{
+ char *endptr;
+
+ /* First, we store the number of currently-existing ComboCIDs. */
+ * (int *) start_address = usedComboCids;
+
+ /* If maxsize is too small, throw an error. */
+ endptr = start_address + sizeof(int) +
+ (sizeof(ComboCidKeyData) * usedComboCids);
+ if (endptr < start_address || endptr > start_address + maxsize)
+ elog(ERROR, "not enough space to serialize ComboCID state");
+
+ /* Now, copy the actual cmin/cmax pairs. */
+ memcpy(start_address + sizeof(int), comboCids,
+ (sizeof(ComboCidKeyData) * usedComboCids));
+}
+
+/*
+ * Read the ComboCID state at the specified address and initialize this
+ * backend with the same ComboCIDs. This is only valid in a backend that
+ * currently has no ComboCIDs (and only makes sense if the transaction state
+ * is serialized and restored as well).
+ */
+void
+RestoreComboCIDState(char *comboCIDstate)
+{
+ int num_elements;
+ ComboCidKeyData *keydata;
+ int i;
+ CommandId cid;
+
+ Assert(!comboCids && !comboHash);
+
+ /* First, we retrieve the number of ComboCIDs that were serialized. */
+ num_elements = * (int *) comboCIDstate;
+ keydata = (ComboCidKeyData *) (comboCIDstate + sizeof(int));
+
+ /* Use GetComboCommandId to restore each ComboCID. */
+ for (i = 0; i < num_elements; i++)
+ {
+ cid = GetComboCommandId(keydata[i].cmin, keydata[i].cmax);
+
+ /* Verify that we got the expected answer. */
+ if (cid != i)
+ elog(ERROR, "unexpected command ID while restoring combo CIDs");
+ }
+}
diff --git a/src/backend/utils/time/snapmgr.c b/src/backend/utils/time/snapmgr.c
index 7cfa0cf..a2cb4a0 100644
--- a/src/backend/utils/time/snapmgr.c
+++ b/src/backend/utils/time/snapmgr.c
@@ -157,6 +157,22 @@ static Snapshot CopySnapshot(Snapshot snapshot);
static void FreeSnapshot(Snapshot snapshot);
static void SnapshotResetXmin(void);
+/*
+ * Snapshot fields to be serialized.
+ *
+ * Only these fields need to be sent to the cooperating backend; the
+ * remaining ones can (and must) set by the receiver upon restore.
+ */
+typedef struct SerializedSnapshotData
+{
+ TransactionId xmin;
+ TransactionId xmax;
+ uint32 xcnt;
+ int32 subxcnt;
+ bool suboverflowed;
+ bool takenDuringRecovery;
+ CommandId curcid;
+} SerializedSnapshotData;
/*
* GetTransactionSnapshot
@@ -188,6 +204,10 @@ GetTransactionSnapshot(void)
Assert(pairingheap_is_empty(&RegisteredSnapshots));
Assert(FirstXactSnapshot == NULL);
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot take query snapshot during a parallel operation");
+
/*
* In transaction-snapshot mode, the first snapshot must live until
* end of xact regardless of what the caller does with it, so we must
@@ -239,6 +259,14 @@ Snapshot
GetLatestSnapshot(void)
{
/*
+ * We might be able to relax this, but nothing that could otherwise work
+ * needs it.
+ */
+ if (IsInParallelMode())
+ elog(ERROR,
+ "cannot update SecondarySnapshot during a parallel operation");
+
+ /*
* So far there are no cases requiring support for GetLatestSnapshot()
* during logical decoding, but it wouldn't be hard to add if required.
*/
@@ -347,7 +375,8 @@ SnapshotSetCommandId(CommandId curcid)
* in GetTransactionSnapshot.
*/
static void
-SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
+SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid,
+ PGPROC *sourceproc)
{
/* Caller should have checked this already */
Assert(!FirstSnapshotSet);
@@ -394,7 +423,15 @@ SetTransactionSnapshot(Snapshot sourcesnap, TransactionId sourcexid)
* doesn't seem worth contorting the logic here to avoid two calls,
* especially since it's not clear that predicate.c *must* do this.
*/
- if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
+ if (sourceproc != NULL)
+ {
+ if (!ProcArrayInstallRestoredXmin(CurrentSnapshot->xmin, sourceproc))
+ ereport(ERROR,
+ (errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
+ errmsg("could not import the requested snapshot"),
+ errdetail("The source transaction is not running anymore.")));
+ }
+ else if (!ProcArrayInstallImportedXmin(CurrentSnapshot->xmin, sourcexid))
ereport(ERROR,
(errcode(ERRCODE_OBJECT_NOT_IN_PREREQUISITE_STATE),
errmsg("could not import the requested snapshot"),
@@ -550,11 +587,24 @@ PushCopiedSnapshot(Snapshot snapshot)
void
UpdateActiveSnapshotCommandId(void)
{
+ CommandId save_curcid, curcid;
Assert(ActiveSnapshot != NULL);
Assert(ActiveSnapshot->as_snap->active_count == 1);
Assert(ActiveSnapshot->as_snap->regd_count == 0);
- ActiveSnapshot->as_snap->curcid = GetCurrentCommandId(false);
+ /*
+ * Don't allow modification of the active snapshot during parallel
+ * operation. We share the snapshot to worker backends at beginning of
+ * parallel operation, so any change to snapshot can lead to
+ * inconsistencies. We have other defenses against
+ * CommandCounterIncrement, but there are a few places that call this
+ * directly, so we put an additional guard here.
+ */
+ save_curcid = ActiveSnapshot->as_snap->curcid;
+ curcid = GetCurrentCommandId(false);
+ if (IsInParallelMode() && save_curcid != curcid)
+ elog(ERROR, "cannot modify commandid in active snapshot during a parallel operation");
+ ActiveSnapshot->as_snap->curcid = curcid;
}
/*
@@ -1289,7 +1339,7 @@ ImportSnapshot(const char *idstr)
errmsg("cannot import a snapshot from a different database")));
/* OK, install the snapshot */
- SetTransactionSnapshot(&snapshot, src_xid);
+ SetTransactionSnapshot(&snapshot, src_xid, NULL);
}
/*
@@ -1393,3 +1443,155 @@ HistoricSnapshotGetTupleCids(void)
Assert(HistoricSnapshotActive());
return tuplecid_data;
}
+
+/*
+ * EstimateSnapshotSpace
+ * Returns the size need to store the given snapshot.
+ *
+ * We are exporting only required fields from the Snapshot, stored in
+ * SerializedSnapshotData.
+ */
+Size
+EstimateSnapshotSpace(Snapshot snap)
+{
+ Size size;
+
+ Assert(snap != InvalidSnapshot);
+ Assert(snap->satisfies == HeapTupleSatisfiesMVCC);
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = add_size(sizeof(SerializedSnapshotData),
+ mul_size(snap->xcnt, sizeof(TransactionId)));
+ if (snap->subxcnt > 0 &&
+ (!snap->suboverflowed || snap->takenDuringRecovery))
+ size = add_size(size,
+ mul_size(snap->subxcnt, sizeof(TransactionId)));
+
+ return size;
+}
+
+/*
+ * SerializeSnapshot
+ * Dumps the serialized snapshot (extracted from given snapshot) onto the
+ * memory location at start_address.
+ */
+void
+SerializeSnapshot(Snapshot snapshot, char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+
+ Assert(snapshot->xcnt >= 0);
+ Assert(snapshot->subxcnt >= 0);
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+
+ /* Copy all required fields */
+ serialized_snapshot->xmin = snapshot->xmin;
+ serialized_snapshot->xmax = snapshot->xmax;
+ serialized_snapshot->xcnt = snapshot->xcnt;
+ serialized_snapshot->subxcnt = snapshot->subxcnt;
+ serialized_snapshot->suboverflowed = snapshot->suboverflowed;
+ serialized_snapshot->takenDuringRecovery = snapshot->takenDuringRecovery;
+ serialized_snapshot->curcid = snapshot->curcid;
+
+ /*
+ * Ignore the SubXID array if it has overflowed, unless the snapshot
+ * was taken during recovey - in that case, top-level XIDs are in subxip
+ * as well, and we mustn't lose them.
+ */
+ if (serialized_snapshot->suboverflowed && !snapshot->takenDuringRecovery)
+ serialized_snapshot->subxcnt = 0;
+
+ /* Copy XID array */
+ if (snapshot->xcnt > 0)
+ memcpy((TransactionId *) (serialized_snapshot + 1),
+ snapshot->xip, snapshot->xcnt * sizeof(TransactionId));
+
+ /*
+ * Copy SubXID array. Don't bother to copy it if it had overflowed,
+ * though, because it's not used anywhere in that case. Except if it's a
+ * snapshot taken during recovery; all the top-level XIDs are in subxip as
+ * well in that case, so we mustn't lose them.
+ */
+ if (snapshot->subxcnt > 0)
+ {
+ Size subxipoff = sizeof(SerializedSnapshotData) +
+ snapshot->xcnt * sizeof(TransactionId);
+
+ memcpy((TransactionId *) ((char *) serialized_snapshot + subxipoff),
+ snapshot->subxip, snapshot->subxcnt * sizeof(TransactionId));
+ }
+}
+
+/*
+ * RestoreSnapshot
+ * Restore a serialized snapshot from the specified address.
+ *
+ * The copy is palloc'd in TopTransactionContext and has initial refcounts set
+ * to 0. The returned snapshot has the copied flag set.
+ */
+Snapshot
+RestoreSnapshot(char *start_address)
+{
+ SerializedSnapshotData *serialized_snapshot;
+ Size size;
+ Snapshot snapshot;
+ TransactionId *serialized_xids;
+
+ serialized_snapshot = (SerializedSnapshotData *) start_address;
+ serialized_xids = (TransactionId *)
+ (start_address + sizeof(SerializedSnapshotData));
+
+ /* We allocate any XID arrays needed in the same palloc block. */
+ size = sizeof(SnapshotData)
+ + serialized_snapshot->xcnt * sizeof(TransactionId)
+ + serialized_snapshot->subxcnt * sizeof(TransactionId);
+
+ /* Copy all required fields */
+ snapshot = (Snapshot) MemoryContextAlloc(TopTransactionContext, size);
+ snapshot->satisfies = HeapTupleSatisfiesMVCC;
+ snapshot->xmin = serialized_snapshot->xmin;
+ snapshot->xmax = serialized_snapshot->xmax;
+ snapshot->xip = NULL;
+ snapshot->xcnt = serialized_snapshot->xcnt;
+ snapshot->subxip = NULL;
+ snapshot->subxcnt = serialized_snapshot->subxcnt;
+ snapshot->suboverflowed = serialized_snapshot->suboverflowed;
+ snapshot->takenDuringRecovery = serialized_snapshot->takenDuringRecovery;
+ snapshot->curcid = serialized_snapshot->curcid;
+
+ /* Copy XIDs, if present. */
+ if (serialized_snapshot->xcnt > 0)
+ {
+ snapshot->xip = (TransactionId *) (snapshot + 1);
+ memcpy(snapshot->xip, serialized_xids,
+ serialized_snapshot->xcnt * sizeof(TransactionId));
+ }
+
+ /* Copy SubXIDs, if present. */
+ if (serialized_snapshot->subxcnt > 0)
+ {
+ snapshot->subxip = snapshot->xip + serialized_snapshot->xcnt;
+ memcpy(snapshot->subxip, serialized_xids + serialized_snapshot->xcnt,
+ serialized_snapshot->subxcnt * sizeof(TransactionId));
+ }
+
+ /* Set the copied flag so that the caller will set refcounts correctly. */
+ snapshot->regd_count = 0;
+ snapshot->active_count = 0;
+ snapshot->copied = true;
+
+ return snapshot;
+}
+
+/*
+ * Install a restored snapshot as the transaction snapshot.
+ *
+ * The second argument is of type void * so that snapmgr.h need not include
+ * the declaration for PGPROC.
+ */
+void
+RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc)
+{
+ SetTransactionSnapshot(snapshot, InvalidTransactionId, master_pgproc);
+}
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
new file mode 100644
index 0000000..8274f84
--- /dev/null
+++ b/src/include/access/parallel.h
@@ -0,0 +1,68 @@
+/*-------------------------------------------------------------------------
+ *
+ * parallel.h
+ * Infrastructure for launching parallel workers
+ *
+ * Portions Copyright (c) 1996-2014, PostgreSQL Global Development Group
+ * Portions Copyright (c) 1994, Regents of the University of California
+ *
+ * src/include/access/parallel.h
+ *
+ *-------------------------------------------------------------------------
+ */
+
+#ifndef PARALLEL_H
+#define PARALLEL_H
+
+#include "access/xlogdefs.h"
+#include "lib/ilist.h"
+#include "postmaster/bgworker.h"
+#include "storage/shm_mq.h"
+#include "storage/shm_toc.h"
+#include "utils/elog.h"
+
+typedef void (*parallel_worker_main_type)(dsm_segment *seg, shm_toc *toc);
+
+typedef struct ParallelWorkerInfo
+{
+ BackgroundWorkerHandle *bgwhandle;
+ shm_mq_handle *error_mqh;
+ int32 pid;
+} ParallelWorkerInfo;
+
+typedef struct ParallelContext
+{
+ dlist_node node;
+ SubTransactionId subid;
+ int nworkers;
+ parallel_worker_main_type entrypoint;
+ char *library_name;
+ char *function_name;
+ ErrorContextCallback *error_context_stack;
+ shm_toc_estimator estimator;
+ dsm_segment *seg;
+ void *private;
+ shm_toc *toc;
+ ParallelWorkerInfo *worker;
+} ParallelContext;
+
+extern bool ParallelMessagePending;
+extern int ParallelWorkerNumber;
+
+#define IsParallelWorker() (ParallelWorkerNumber >= 0)
+
+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 *);
+extern void LaunchParallelWorkers(ParallelContext *);
+extern void WaitForParallelWorkersToFinish(ParallelContext *);
+extern void DestroyParallelContext(ParallelContext *);
+extern bool ParallelContextActive(void);
+
+extern void HandleParallelMessageInterrupt(void);
+extern void HandleParallelMessages(void);
+extern void AtEOXact_Parallel(bool isCommit);
+extern void AtEOSubXact_Parallel(bool isCommit, SubTransactionId mySubId);
+extern void ParallelWorkerReportLastRecEnd(XLogRecPtr);
+
+#endif /* PARALLEL_H */
diff --git a/src/include/access/xact.h b/src/include/access/xact.h
index 8da6aa9..90c1e12 100644
--- a/src/include/access/xact.h
+++ b/src/include/access/xact.h
@@ -78,9 +78,12 @@ extern bool MyXactAccessedTempRel;
typedef enum
{
XACT_EVENT_COMMIT,
+ XACT_EVENT_PARALLEL_COMMIT,
XACT_EVENT_ABORT,
+ XACT_EVENT_PARALLEL_ABORT,
XACT_EVENT_PREPARE,
XACT_EVENT_PRE_COMMIT,
+ XACT_EVENT_PARALLEL_PRE_COMMIT,
XACT_EVENT_PRE_PREPARE
} XactEvent;
@@ -321,6 +324,10 @@ extern void BeginInternalSubTransaction(char *name);
extern void ReleaseCurrentSubTransaction(void);
extern void RollbackAndReleaseCurrentSubTransaction(void);
extern bool IsSubTransaction(void);
+extern Size EstimateTransactionStateSpace(void);
+extern void SerializeTransactionState(Size maxsize, char *start_address);
+extern void StartParallelWorkerTransaction(char *tstatespace);
+extern void EndParallelWorkerTransaction(void);
extern bool IsTransactionBlock(void);
extern bool IsTransactionOrTransactionBlock(void);
extern char TransactionBlockStatusCode(void);
@@ -357,4 +364,8 @@ extern const char *xact_identify(uint8 info);
extern void ParseCommitRecord(uint8 info, xl_xact_commit *xlrec, xl_xact_parsed_commit *parsed);
extern void ParseAbortRecord(uint8 info, xl_xact_abort *xlrec, xl_xact_parsed_abort *parsed);
+extern void EnterParallelMode(void);
+extern void ExitParallelMode(void);
+extern bool IsInParallelMode(void);
+
#endif /* XACT_H */
diff --git a/src/include/catalog/namespace.h b/src/include/catalog/namespace.h
index cf5f7d0..f3b005f 100644
--- a/src/include/catalog/namespace.h
+++ b/src/include/catalog/namespace.h
@@ -140,7 +140,7 @@ extern Oid FindDefaultConversionProc(int32 for_encoding, int32 to_encoding);
/* initialization & transaction cleanup code */
extern void InitializeSearchPath(void);
-extern void AtEOXact_Namespace(bool isCommit);
+extern void AtEOXact_Namespace(bool isCommit, bool parallel);
extern void AtEOSubXact_Namespace(bool isCommit, SubTransactionId mySubid,
SubTransactionId parentSubid);
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 418f6aa..b9a5c40 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -642,6 +642,9 @@ extern PGFunction load_external_function(char *filename, char *funcname,
extern PGFunction lookup_external_function(void *filehandle, char *funcname);
extern void load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName);
+extern Size EstimateLibraryStateSpace(void);
+extern void SerializeLibraryState(Size maxsize, char *start_address);
+extern void RestoreLibraryState(char *start_address);
/*
* Support for aggregate functions
diff --git a/src/include/libpq/pqmq.h b/src/include/libpq/pqmq.h
index 5f2815c..ad7589d 100644
--- a/src/include/libpq/pqmq.h
+++ b/src/include/libpq/pqmq.h
@@ -17,6 +17,7 @@
#include "storage/shm_mq.h"
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 void pq_parse_errornotice(StringInfo str, ErrorData *edata);
diff --git a/src/include/miscadmin.h b/src/include/miscadmin.h
index eacfccb..c389939 100644
--- a/src/include/miscadmin.h
+++ b/src/include/miscadmin.h
@@ -271,6 +271,7 @@ extern void check_stack_depth(void);
/* in tcop/utility.c */
extern void PreventCommandIfReadOnly(const char *cmdname);
+extern void PreventCommandIfParallelMode(const char *cmdname);
extern void PreventCommandDuringRecovery(const char *cmdname);
/* in utils/misc/guc.c */
diff --git a/src/include/postmaster/bgworker.h b/src/include/postmaster/bgworker.h
index a81b90b..de9180d 100644
--- a/src/include/postmaster/bgworker.h
+++ b/src/include/postmaster/bgworker.h
@@ -112,6 +112,8 @@ extern BgwHandleStatus GetBackgroundWorkerPid(BackgroundWorkerHandle *handle,
extern BgwHandleStatus
WaitForBackgroundWorkerStartup(BackgroundWorkerHandle *
handle, pid_t *pid);
+extern BgwHandleStatus
+WaitForBackgroundWorkerShutdown(BackgroundWorkerHandle *);
/* Terminate a bgworker */
extern void TerminateBackgroundWorker(BackgroundWorkerHandle *handle);
diff --git a/src/include/storage/procarray.h b/src/include/storage/procarray.h
index 97c6e93..a9b40ed 100644
--- a/src/include/storage/procarray.h
+++ b/src/include/storage/procarray.h
@@ -46,6 +46,7 @@ extern Snapshot GetSnapshotData(Snapshot snapshot);
extern bool ProcArrayInstallImportedXmin(TransactionId xmin,
TransactionId sourcexid);
+extern bool ProcArrayInstallRestoredXmin(TransactionId xmin, PGPROC *proc);
extern RunningTransactions GetRunningTransactionData(void);
diff --git a/src/include/storage/procsignal.h b/src/include/storage/procsignal.h
index ac9d236..af1a0cd 100644
--- a/src/include/storage/procsignal.h
+++ b/src/include/storage/procsignal.h
@@ -31,6 +31,7 @@ typedef enum
{
PROCSIG_CATCHUP_INTERRUPT, /* sinval catchup interrupt */
PROCSIG_NOTIFY_INTERRUPT, /* listen/notify interrupt */
+ PROCSIG_PARALLEL_MESSAGE, /* message from cooperating parallel backend */
/* Recovery conflict reasons */
PROCSIG_RECOVERY_CONFLICT_DATABASE,
diff --git a/src/include/utils/combocid.h b/src/include/utils/combocid.h
index ce7b47c..f2faa12 100644
--- a/src/include/utils/combocid.h
+++ b/src/include/utils/combocid.h
@@ -21,5 +21,8 @@
*/
extern void AtEOXact_ComboCid(void);
+extern void RestoreComboCIDState(char *comboCIDstate);
+extern void SerializeComboCIDState(Size maxsize, char *start_address);
+extern Size EstimateComboCIDStateSpace(void);
#endif /* COMBOCID_H */
diff --git a/src/include/utils/snapmgr.h b/src/include/utils/snapmgr.h
index 64d2ec1..f8524eb 100644
--- a/src/include/utils/snapmgr.h
+++ b/src/include/utils/snapmgr.h
@@ -64,4 +64,9 @@ extern void SetupHistoricSnapshot(Snapshot snapshot_now, struct HTAB *tuplecids)
extern void TeardownHistoricSnapshot(bool is_error);
extern bool HistoricSnapshotActive(void);
+extern Size EstimateSnapshotSpace(Snapshot snapshot);
+extern void SerializeSnapshot(Snapshot snapshot, char *start_address);
+extern Snapshot RestoreSnapshot(char *start_address);
+extern void RestoreTransactionSnapshot(Snapshot snapshot, void *master_pgproc);
+
#endif /* SNAPMGR_H */
--
1.7.9.6 (Apple Git-31.1)
parallel-mode-locking.patchbinary/octet-stream; name=parallel-mode-locking.patchDownload
From f6350fe3f7a8b989c8fa06d47251cd0930ca9b8e Mon Sep 17 00:00:00 2001
From: Robert Haas <rhaas@postgresql.org>
Date: Tue, 28 Apr 2015 17:23:47 -0400
Subject: [PATCH 2/2] Lock management changes for parallelism.
This part still needs more work.
---
src/backend/access/transam/README.parallel | 70 ++++++++-
src/backend/access/transam/parallel.c | 22 ++-
src/backend/access/transam/xact.c | 4 +
src/backend/storage/ipc/standby.c | 6 +-
src/backend/storage/lmgr/lock.c | 233 +++++++++++++++++++++++++++-
src/include/storage/lock.h | 10 +-
6 files changed, 333 insertions(+), 12 deletions(-)
diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
index 1005186..2257e4c 100644
--- a/src/backend/access/transam/README.parallel
+++ b/src/backend/access/transam/README.parallel
@@ -122,9 +122,8 @@ worker. This includes:
values are restored, this incidentally sets SessionUserId and OuterUserId
to the correct values. This final step restores CurrentUserId.
-To prevent undetected or unprincipled deadlocks when running in parallel mode,
-this could should eventually handle heavyweight locks in some way. This is
-not implemented yet.
+ - Locks on relations and database objects, and any lock held on the
+ transaction's XID. See lock management, below.
Transaction Integration
=======================
@@ -188,6 +187,71 @@ importantly:
safely access the initiating backend's pg_temp namespace, and should
not create one of their own.
+Lock Management
+===============
+
+Certain heavyweight locks that the initiating backend holds at the beginning
+of parallelism are copied to each worker, which unconditionally acquires them.
+The parallel backends acquire - without waiting - each such lock that the
+leader holds, even if that lock is self-exclusive. This creates the unusual
+situation that a lock which could normally only be held by a single backend
+can be shared among several backends in a parallel group.
+
+Obviously, this presents significant hazards that would not be present in
+normal execution. If, for example, a backend were to initiate parallelism
+while ReindexIsProcessingIndex() were true for some index, the parallel
+backends launched at that time would neither share this state nor be excluded
+from accessing the index via the heavyweight lock mechanism. It is therefore
+imperative that backends only initiate parallelism from places where it will
+be safe for parallel workers to access the relations on which they hold locks.
+It is also important that they not afterwards do anything which causes access
+to those relations to become unsafe, or at least not until after parallelism
+has concluded. The fact that parallelism is strictly read-only means that the
+opportunities for such mishaps are few and far between; furthermore, most
+operations which present these hazards are DDL operations, which will be
+rejected by CheckTableNotInUse() if parallel mode is active.
+
+Only relation locks, locks on database or shared objects, and any lock on
+our transaction ID are copied to workers. Advisory locks are not copied,
+but the leader may hold them at the start of parallelism; they cannot
+subsequently be manipulated while parallel mode is active. It is not safe
+to attempt parallelism while holding a lock of any other type, such as a
+page, tuple, or relation extension lock, and such attempts will fail. Such
+locks can be taken and released during parallel mode; they merely cannot be
+held at the start of parallel mode, since we would then fail to provide
+necessary mutual exclusion.
+
+Copying locks to workers is important for the avoidance of undetected
+deadlock between the initiating process and its parallel workers. If the
+initiating process holds a lock on an object at the start of parallelism,
+and the worker subsequently attempts to acquire a lock on that same object
+and blocks, this will result in an undetected deadlock, because the
+initiating process cannot finish the transaction (thus releasing the lock)
+until the worker terminates, and the worker cannot acquire the lock while
+the initiating process holds it. Locks which the processes involved acquire
+and then release during parallelism do not present this hazard; they simply
+force the processes involved to take turns accessing the protected resource.
+
+Locks which the processes involved acquire and retain *do* present this
+hazard. In general, we can avoid this problem by forbidding any of the
+group of parallel processes from holding any heavyweight locks at the end of
+parallelism which they did not hold at the start of parallelism. For now,
+we take this approach; if it proves to be onerous in the future, we can look
+for ways to relax it then.
+
+Copying locks to workers is also important for the avoidance of undetected
+deadlock involving both the parallel processes and other processes. For
+example, suppose processes A1 and A2 are cooperating parallel processes and
+B is an unrelated process. Suppose A1 holds a lock L1 and waits for A2 to
+finish; B holds a lock L2 and blocks waiting for L1; and A2 blocks waiting
+for L2. Without lock copying, the waits-for graph is A2 -> B -> A1; there
+is no cycle. With lock copying, A2 will also hold the lock on L1 and the
+deadlock detector can find the cycle A2 -> B -> A2. As in the case of
+deadlocks within the parallel group, undetected deadlock occur if either A1
+or A2 acquired a lock after the start of parallelism and attempted to
+retain it beyond the end of parallelism. The prohibitions discussed above
+protect us against this case.
+
Coding Conventions
===================
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index 8ed7314..d2cec8e 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -56,7 +56,8 @@
#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
-#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_LOCK UINT64CONST(0xFFFFFFFFFFFF0009)
+#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF000A)
/* Fixed-size parallel state. */
typedef struct FixedParallelState
@@ -191,6 +192,7 @@ InitializeParallelDSM(ParallelContext *pcxt)
Size tsnaplen = 0;
Size asnaplen = 0;
Size tstatelen = 0;
+ Size lockstatelen = 0;
Size segsize = 0;
int i;
FixedParallelState *fps;
@@ -223,8 +225,10 @@ InitializeParallelDSM(ParallelContext *pcxt)
shm_toc_estimate_chunk(&pcxt->estimator, asnaplen);
tstatelen = EstimateTransactionStateSpace();
shm_toc_estimate_chunk(&pcxt->estimator, tstatelen);
+ lockstatelen = EstimateLockStateSpace();
+ shm_toc_estimate_chunk(&pcxt->estimator, lockstatelen);
/* If you add more chunks here, you probably need to add keys. */
- shm_toc_estimate_keys(&pcxt->estimator, 6);
+ shm_toc_estimate_keys(&pcxt->estimator, 7);
/* Estimate space need for error queues. */
StaticAssertStmt(BUFFERALIGN(PARALLEL_ERROR_QUEUE_SIZE) ==
@@ -295,6 +299,7 @@ InitializeParallelDSM(ParallelContext *pcxt)
char *tsnapspace;
char *asnapspace;
char *tstatespace;
+ char *lockstatespace;
char *error_queue_space;
/* Serialize shared libraries we have loaded. */
@@ -326,6 +331,11 @@ InitializeParallelDSM(ParallelContext *pcxt)
SerializeTransactionState(tstatelen, tstatespace);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_TRANSACTION_STATE, tstatespace);
+ /* Serialize heavyweight locks. */
+ lockstatespace = shm_toc_allocate(pcxt->toc, lockstatelen);
+ SerializeLockState(lockstatelen, lockstatespace);
+ shm_toc_insert(pcxt->toc, PARALLEL_KEY_LOCK, lockstatespace);
+
/* Allocate space for worker information. */
pcxt->worker = palloc0(sizeof(ParallelWorkerInfo) * pcxt->nworkers);
@@ -452,6 +462,8 @@ LaunchParallelWorkers(ParallelContext *pcxt)
void
WaitForParallelWorkersToFinish(ParallelContext *pcxt)
{
+ CheckForRetainedParallelLocks();
+
for (;;)
{
bool anyone_alive = false;
@@ -807,6 +819,7 @@ ParallelWorkerMain(Datum main_arg)
char *tsnapspace;
char *asnapspace;
char *tstatespace;
+ char *lockstatespace;
StringInfoData msgbuf;
/* Establish signal handlers. */
@@ -926,6 +939,11 @@ ParallelWorkerMain(Datum main_arg)
/* Restore user ID and security context. */
SetUserIdAndSecContext(fps->current_user_id, fps->sec_context);
+ /* Restore locks. */
+ lockstatespace = shm_toc_lookup(toc, PARALLEL_KEY_LOCK);
+ Assert(lockstatespace != NULL);
+ RestoreLockState(fps->parallel_master_pgproc, lockstatespace);
+
/*
* We've initialized all of our state now; nothing should change hereafter.
*/
diff --git a/src/backend/access/transam/xact.c b/src/backend/access/transam/xact.c
index ebc232a..f8b529e 100644
--- a/src/backend/access/transam/xact.c
+++ b/src/backend/access/transam/xact.c
@@ -893,6 +893,8 @@ ExitParallelMode(void)
Assert(s->parallelModeLevel > 1 || !ParallelContextActive());
--s->parallelModeLevel;
+
+ CheckForRetainedParallelLocks();
}
/*
@@ -1950,6 +1952,7 @@ CommitTransaction(void)
/* If we might have parallel workers, clean them up now. */
if (IsInParallelMode())
{
+ CheckForRetainedParallelLocks();
AtEOXact_Parallel(true);
s->parallelModeLevel = 0;
}
@@ -2483,6 +2486,7 @@ AbortTransaction(void)
/* If in parallel mode, clean up workers and exit parallel mode. */
if (IsInParallelMode())
{
+ ForgetParallelLocks();
AtEOXact_Parallel(false);
s->parallelModeLevel = 0;
}
diff --git a/src/backend/storage/ipc/standby.c b/src/backend/storage/ipc/standby.c
index 292bed5..6002d51 100644
--- a/src/backend/storage/ipc/standby.c
+++ b/src/backend/storage/ipc/standby.c
@@ -366,7 +366,8 @@ ResolveRecoveryConflictWithLock(Oid dbOid, Oid relOid)
ResolveRecoveryConflictWithVirtualXIDs(backends,
PROCSIG_RECOVERY_CONFLICT_LOCK);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
!= LOCKACQUIRE_NOT_AVAIL)
lock_acquired = true;
}
@@ -592,7 +593,8 @@ StandbyAcquireAccessExclusiveLock(TransactionId xid, Oid dbOid, Oid relOid)
*/
SET_LOCKTAG_RELATION(locktag, newlock->dbOid, newlock->relOid);
- if (LockAcquireExtended(&locktag, AccessExclusiveLock, true, true, false)
+ if (LockAcquireExtended(&locktag, AccessExclusiveLock,
+ true, true, false, false)
== LOCKACQUIRE_NOT_AVAIL)
ResolveRecoveryConflictWithLock(newlock->dbOid, newlock->relOid);
}
diff --git a/src/backend/storage/lmgr/lock.c b/src/backend/storage/lmgr/lock.c
index 1eb2d4b..b19bffa 100644
--- a/src/backend/storage/lmgr/lock.c
+++ b/src/backend/storage/lmgr/lock.c
@@ -35,6 +35,7 @@
#include "access/transam.h"
#include "access/twophase.h"
#include "access/twophase_rmgr.h"
+#include "access/xact.h"
#include "access/xlog.h"
#include "miscadmin.h"
#include "pg_trace.h"
@@ -258,6 +259,13 @@ static LOCALLOCK *StrongLockInProgress;
static LOCALLOCK *awaitedLock;
static ResourceOwner awaitedOwner;
+/*
+ * Track locks acquired in parallel mode. Any such locks must be released
+ * before exiting parallel mode; see src/backend/access/transam/README.parallel
+ */
+static LOCALLOCKTAG *LocksAcquiredInParallelMode;
+static int nLocksAcquiredInParallelMode;
+static int maxLocksAcquiredInParallelMode;
#ifdef LOCK_DEBUG
@@ -669,7 +677,8 @@ LockAcquire(const LOCKTAG *locktag,
bool sessionLock,
bool dontWait)
{
- return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true);
+ return LockAcquireExtended(locktag, lockmode, sessionLock, dontWait, true,
+ NULL);
}
/*
@@ -680,13 +689,20 @@ LockAcquire(const LOCKTAG *locktag,
* caller to note that the lock table is full and then begin taking
* extreme action to reduce the number of other lock holders before
* retrying the action.
+ *
+ * leader_proc should be false except for the case of a parallel worker
+ * reacquiring locks already held by the parallel group leader. In that
+ * case, we never log the lock acquisition since the parent has already
+ * done it; and more importantly and surprisingly, we ignore lock conflicts.
+ * See src/backend/access/transam/README.parallel for further discussion.
*/
LockAcquireResult
LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool reportMemoryError)
+ bool reportMemoryError,
+ PGPROC *leader_proc)
{
LOCKMETHODID lockmethodid = locktag->locktag_lockmethodid;
LockMethod lockMethodTable;
@@ -700,6 +716,7 @@ LockAcquireExtended(const LOCKTAG *locktag,
LWLock *partitionLock;
int status;
bool log_lock = false;
+ bool remember_parallel_lock = false;
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
@@ -797,13 +814,42 @@ LockAcquireExtended(const LOCKTAG *locktag,
if (lockmode >= AccessExclusiveLock &&
locktag->locktag_type == LOCKTAG_RELATION &&
!RecoveryInProgress() &&
- XLogStandbyInfoActive())
+ XLogStandbyInfoActive() && leader_proc == NULL)
{
LogAccessExclusiveLockPrepare();
log_lock = true;
}
/*
+ * If we're in parallel mode, ensure that there will be space to
+ * remember this lock, so that we can later check that it got released
+ * before the end of parallelism. We can skip this when reacquiring the
+ * parallel leader's locks, as those don't have to be released before
+ * parallelism ends. See src/backend/access/transam/README.parallel for
+ * more details.
+ */
+ if (leader_proc == NULL && IsInParallelMode())
+ {
+ if (maxLocksAcquiredInParallelMode == 0)
+ {
+ LocksAcquiredInParallelMode =
+ MemoryContextAlloc(TopMemoryContext, 8 * sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode = 8;
+ }
+ else if (nLocksAcquiredInParallelMode >=
+ maxLocksAcquiredInParallelMode)
+ {
+ LocksAcquiredInParallelMode =
+ repalloc(LocksAcquiredInParallelMode,
+ 2 * maxLocksAcquiredInParallelMode *
+ sizeof(LOCALLOCKTAG));
+ maxLocksAcquiredInParallelMode *= 2;
+ }
+ Assert(nLocksAcquiredInParallelMode < maxLocksAcquiredInParallelMode);
+ remember_parallel_lock = true;
+ }
+
+ /*
* Attempt to take lock via fast path, if eligible. But if we remember
* having filled up the fast path array, we don't attempt to make any
* further use of it until we release some locks. It's possible that some
@@ -842,6 +888,9 @@ LockAcquireExtended(const LOCKTAG *locktag,
locallock->lock = NULL;
locallock->proclock = NULL;
GrantLockLocal(locallock, owner);
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] =
+ localtag;
return LOCKACQUIRE_OK;
}
}
@@ -910,7 +959,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
/*
* If lock requested conflicts with locks requested by waiters, must join
* wait queue. Otherwise, check for conflict with already-held locks.
- * (That's last because most complex check.)
+ * (That's last because most complex check.) Parallel reacquire never
+ * conflicts.
*/
if (lockMethodTable->conflictTab[lockmode] & lock->waitMask)
status = STATUS_FOUND;
@@ -918,6 +968,26 @@ LockAcquireExtended(const LOCKTAG *locktag,
status = LockCheckConflicts(lockMethodTable, lockmode,
lock, proclock);
+ /*
+ * When reacquiring the master's locks in a parallel worker, we ignore
+ * lock conflicts; in effect, the parallel leader and the worker are
+ * sharing the lock. However, if the leader somehow manages to die before
+ * we reach this point, then we wouldn't be sharing the lock with the
+ * parallel group leader - we'd just be grabbing it in the face of
+ * conflicts. To make sure that can't happen, check that the leader
+ * still has the lock.
+ */
+ if (status == STATUS_FOUND && leader_proc != NULL)
+ {
+ PROCLOCKTAG leadertag;
+
+ leadertag.myLock = lock;
+ leadertag.myProc = MyProc;
+ if (hash_search(LockMethodProcLockHash, (void *) &leadertag,
+ HASH_FIND, NULL))
+ status = STATUS_OK;
+ }
+
if (status == STATUS_OK)
{
/* No conflict with held or previously requested locks */
@@ -1034,6 +1104,8 @@ LockAcquireExtended(const LOCKTAG *locktag,
locktag->locktag_field2);
}
+ if (remember_parallel_lock)
+ LocksAcquiredInParallelMode[nLocksAcquiredInParallelMode++] = localtag;
return LOCKACQUIRE_OK;
}
@@ -1818,6 +1890,38 @@ LockRelease(const LOCKTAG *locktag, LOCKMODE lockmode, bool sessionLock)
}
/*
+ * If we're tracking locks acquired in parallel mode and we're about
+ * to release the last copy of this lock, remove it from our tracking
+ * table. We search back-to-front since we'll probably release the last
+ * lock acquired first.
+ */
+ if (IsInParallelMode() && locallock->nLocks == 1)
+ {
+ int n;
+ bool found = false;
+
+ for (n = nLocksAcquiredInParallelMode - 1; n >= 0; --n)
+ {
+ LOCALLOCKTAG *ptag = &LocksAcquiredInParallelMode[n];
+
+ if (memcmp(ptag, &localtag, sizeof(LOCALLOCKTAG)) == 0)
+ {
+ --nLocksAcquiredInParallelMode;
+ if (n < nLocksAcquiredInParallelMode)
+ memmove(ptag, ptag + 1, sizeof(LOCALLOCKTAG) *
+ (nLocksAcquiredInParallelMode - n));
+ found = true;
+ break;
+ }
+ }
+
+ if (!found)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot release locks acquired before the start of parallelism while parallelism is active")));
+ }
+
+ /*
* Decrease the total local count. If we're still holding the lock, we're
* done.
*/
@@ -1943,6 +2047,7 @@ LockReleaseAll(LOCKMETHODID lockmethodid, bool allLocks)
if (lockmethodid <= 0 || lockmethodid >= lengthof(LockMethods))
elog(ERROR, "unrecognized lock method: %d", lockmethodid);
lockMethodTable = LockMethods[lockmethodid];
+ nLocksAcquiredInParallelMode = 0;
#ifdef LOCK_DEBUG
if (*(lockMethodTable->trace_flag))
@@ -3557,6 +3662,126 @@ GetLockmodeName(LOCKMETHODID lockmethodid, LOCKMODE mode)
return LockMethods[lockmethodid]->lockModeNames[mode];
}
+/*
+ * Estimate the amount of space required to record information on locks that
+ * need to be copied to parallel workers.
+ */
+Size
+EstimateLockStateSpace(void)
+{
+ return add_size(sizeof(long),
+ mul_size(hash_get_num_entries(LockMethodLocalHash),
+ sizeof(LOCALLOCKTAG)));
+}
+
+/*
+ * Serialize relevant heavyweight lock state into the memory beginning at
+ * start_address. maxsize should be at least as large as the value returned
+ * by EstimateLockStateSpace.
+ */
+void
+SerializeLockState(Size maxsize, char *start_address)
+{
+ char *endptr = start_address + maxsize;
+ char *curptr = start_address + sizeof(long);
+ HASH_SEQ_STATUS status;
+ LOCALLOCK *locallock;
+ long count = 0;
+
+ hash_seq_init(&status, LockMethodLocalHash);
+
+ while ((locallock = (LOCALLOCK *) hash_seq_search(&status)) != NULL)
+ {
+ LockTagType type;
+
+ if (locallock->nLocks == 0)
+ continue;
+
+ /*
+ * We only copy ordinary heavyweight locks; advisory lock operations
+ * are prohibited while in parallel mode.
+ */
+ if (locallock->tag.lock.locktag_lockmethodid != DEFAULT_LOCKMETHOD)
+ continue;
+
+ type = locallock->tag.lock.locktag_type;
+
+ /*
+ * Let's make sure that the parallel leader isn't holding some kind
+ * of lock we're not expecting, like a relation extension lock, or
+ * a page or tuple lock. If we are, this is probably not a safe point
+ * from which to initiate parallelism.
+ */
+ if (type == LOCKTAG_TRANSACTION)
+ {
+ if (locallock->tag.mode != ExclusiveLock)
+ elog(ERROR, "unexpected non-exclusive transaction lock");
+ }
+ else if (type != LOCKTAG_RELATION && type != LOCKTAG_OBJECT)
+ elog(ERROR, "unexpected lock tag type: %d", (int) type);
+
+ /* Double-check that we're not going to overrun the space. */
+ if (curptr >= endptr)
+ elog(ERROR, "not enough space to serialize lock state");
+
+ memcpy(curptr, &locallock->tag, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ count++;
+ }
+
+ memcpy(start_address, &count, sizeof(long));
+}
+
+/*
+ * Retake the locals specified by the serialized lock state.
+ */
+void
+RestoreLockState(PGPROC *leader_proc, char *start_address)
+{
+ char *curptr = start_address + sizeof(long);
+ long count;
+
+ Assert(leader_proc != NULL);
+ memcpy(&count, start_address, sizeof(long));
+
+ while (count > 0)
+ {
+ LOCALLOCKTAG locallocktag;
+ LockAcquireResult result;
+
+ memcpy(&locallocktag, curptr, sizeof(LOCALLOCKTAG));
+ curptr += sizeof(LOCALLOCKTAG);
+ --count;
+
+ result = LockAcquireExtended(&locallocktag.lock, locallocktag.mode,
+ false, true, true, leader_proc);
+ if (result != LOCKACQUIRE_OK)
+ ereport(ERROR,
+ (errmsg("parallel worker lock not available")));
+ }
+}
+
+/*
+ * Verify that no locks acquired while in parallel mode are still held.
+ */
+void
+CheckForRetainedParallelLocks(void)
+{
+ if (nLocksAcquiredInParallelMode > 0)
+ ereport(ERROR,
+ (errcode(ERRCODE_INVALID_TRANSACTION_STATE),
+ errmsg("cannot retain locks acquired while in parallel mode")));
+}
+
+/*
+ * Stop tracking locks acquired in parallel mode.
+ */
+void
+ForgetParallelLocks(void)
+{
+ nLocksAcquiredInParallelMode = 0;
+}
+
#ifdef LOCK_DEBUG
/*
* Dump all locks in the given proc's myProcLocks lists.
diff --git a/src/include/storage/lock.h b/src/include/storage/lock.h
index dae517f..80144b9 100644
--- a/src/include/storage/lock.h
+++ b/src/include/storage/lock.h
@@ -503,7 +503,8 @@ extern LockAcquireResult LockAcquireExtended(const LOCKTAG *locktag,
LOCKMODE lockmode,
bool sessionLock,
bool dontWait,
- bool report_memory_error);
+ bool report_memory_error,
+ PGPROC *leader_proc);
extern void AbortStrongLockAcquire(void);
extern bool LockRelease(const LOCKTAG *locktag,
LOCKMODE lockmode, bool sessionLock);
@@ -564,4 +565,11 @@ extern void VirtualXactLockTableInsert(VirtualTransactionId vxid);
extern void VirtualXactLockTableCleanup(void);
extern bool VirtualXactLock(VirtualTransactionId vxid, bool wait);
+/* Parallel worker state sharing. */
+extern Size EstimateLockStateSpace(void);
+extern void SerializeLockState(Size maxsize, char *start_address);
+extern void RestoreLockState(PGPROC *leader_proc, char *start_address);
+extern void CheckForRetainedParallelLocks(void);
+extern void ForgetParallelLocks(void);
+
#endif /* LOCK_H */
--
1.7.9.6 (Apple Git-31.1)
On Wed, Apr 29, 2015 at 12:23 PM, Robert Haas <robertmhaas@gmail.com> wrote:
So, I think it makes sense to split up this patch in two. There's no
real debate, AFAICS, about anything in the patch other than the
heavyweight locking stuff. So I'd like to go ahead and commit the
rest. That's attached here as parallel-mode-v10.patch.
Hearing no objections, done.
Still hoping for some input on the rest.
--
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 21, 2015 at 02:08:00PM -0400, Robert Haas wrote:
On Tue, Mar 24, 2015 at 3:26 PM, Andres Freund <andres@2ndquadrant.com> wrote:
[proposal: teach deadlock detector about parallel master waiting on workers]
deadlock.c is far from simple, and at least I don't find the control
flow to be particularly clear. So it's not easy. It'd be advantageous
to tackle things at that level because it'd avoid the need to acquire
locks on some lock's waitqueue when blocking; we're going to do that a
lot.But It seems to me that it should be possible to suceed: In
FindLockCycleRecurse(), in the case that we're not waiting for an actual
lock (checkProc->links.next == NULL) we can add a case that considers
the 'blocking parallelism' case. ISTM that that's just a
FindLockCycleRecurse() call on the process that we're waiting for. We'd
either have to find the other side's locktag for DEADLOCK_INFO or invent
another field in there; but that seems like a solveable problem.
1. The master doesn't actually *wait* until the very end of the parallel phase.
2. When the master does wait, it waits for all of the parallel workers
at once, not each one individually.So, I don't think anything as simplistic as teaching a blocking
shm_mq_receive() to tip off the deadlock detector that we're waiting
for the process on the other end of that particular queue can ever
work. Because of point #2, that never happens. When I first started
thinking about how to fix this, I said, well, that's no big deal, we
can just advertise the whole list of processes that we're waiting for
in shared memory, rather than just the one. This is a bit tricky,
though. Any general API for any backend to advertise that it's waiting
for an arbitrary subset of the other backends would require O(n^2)
shared memory state. That wouldn't be completely insane, but it's not
great, either. For this particular case, we could optimize that down
to O(n) by just chaining all of the children of a given parallel group
leader in a linked whose nodes are inlined in their PGPROCs, but that
doesn't feel very general, because it forecloses the possibility of
the children ever using that API, and I think they might need to. If
nothing else, they might need to advertise that they're blocking on
the master if they are trying to send data, the queue is full, and
they have to wait for the master to drain some of it before they can
proceed.
Perhaps, rather than model it as master M waiting on worker list W1|W2|W3,
model it with queue-nonempty and queue-nonfull events, one pair per queue. M
subscribes to queue0-nonempty, and each of W1,W2,W3 publish to it. M
publishes to queue0-nonfull, and the workers subscribe. An edge forms in the
deadlock detector's graph when all of an event's subscribers wait on it. (One
can approximate that in userspace with a pair of advisory locks.)
After thinking about it a bit more, I realized that even if we settle
on some solution to that problem, there's another issues: the
wait-edges created by this system don't really have the same semantics
as regular lock waits. Suppose A is waiting on a lock held by B and B
is waiting for a lock held by A; that's a deadlock. But if A is
waiting for B to write to a tuple queue and B is waiting for A to read
from a tuple queue, that's not a deadlock if the queues in question
are the same.
I do see a deadlock. B wants to block until A reads from the queue, so it
advertises that and sleeps. Waking up deadlock_timeout later, it notices that
A is waiting for B to write something. B will not spontaneously suspend
waiting to write to the queue, nor will A suspend waiting to read from the
queue. Thus, the deadlock is valid. This assumes the deadlock detector
reasons from an authoritative picture of pending waits and that we reliably
wake up a process when the condition it sought has arrived. We have that for
heavyweight lock acquisitions. The assumption may be incompatible with
Andres's hope, quoted above, of avoiding the full lock acquisition procedure.
A further difference in the semantics of these wait-edges is that if
process A is awaiting AccessExclusiveLock on a resource held by B, C,
and D at AccessShareLock, it needs to wait for all of those processes
to release their locks before it can do anything else. On the other
hand, if process A is awaiting tuples from B, C, and D, it just needs
ONE of those processes to emit tuples in order to make progress. Now
Right.
maybe that doesn't make any difference in practice, because even if
two of those processes are making lively progress and A is receiving
tuples from them and processing them like gangbusters, that's probably
not going to help the third one get unstuck. If we adopt the approach
of discounting that possibility, then as long as the parallel leader
can generate tuples locally (that is, local_scan_done = false) we
don't report the deadlock, but as soon as it can no longer do that
(local_scan_done = true) then we do, even though we could still
theoretically read more tuples from the non-stuck workers.
I can't discount the possibility that the master could unstick a worker in the
course of ingesting tuples from other workers. We could accept a coding rule
against parallel algorithms behaving that way, on pain of unprincipled
deadlock. It's unattractive to bake such a high-level notion of acceptable
wait patterns into the deadlock detector, and every coding rule has noteworthy
cost. Since detecting deadlocks long after they were inevitable is also
unattractive, the rule might be worthwhile.
So then
you have to wonder why we're not solving problem #1, because the
deadlock was just as certain before we generated the maximum possible
number of tuples locally as it was afterwards.
The "1." above reads like a benefit, not a problem. What might you solve
about it?
Thanks,
nm
--
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, May 2, 2015 at 12:35 PM, Noah Misch <noah@leadboat.com> wrote:
Perhaps, rather than model it as master M waiting on worker list W1|W2|W3,
model it with queue-nonempty and queue-nonfull events, one pair per queue.
Each individual queue has only a single reader and a single writer.
In your example, there would be three queues, not just one. Of
course, one could design a new shared memory data structure
representing a collection of related queues, but I still don't see
exactly how we get around the problem of that requiring
O(MaxBackends^2) storage space.
M
subscribes to queue0-nonempty, and each of W1,W2,W3 publish to it. M
publishes to queue0-nonfull, and the workers subscribe. An edge forms in the
deadlock detector's graph when all of an event's subscribers wait on it. (One
can approximate that in userspace with a pair of advisory locks.)
An edge between which two processes?
After thinking about it a bit more, I realized that even if we settle
on some solution to that problem, there's another issues: the
wait-edges created by this system don't really have the same semantics
as regular lock waits. Suppose A is waiting on a lock held by B and B
is waiting for a lock held by A; that's a deadlock. But if A is
waiting for B to write to a tuple queue and B is waiting for A to read
from a tuple queue, that's not a deadlock if the queues in question
are the same.I do see a deadlock. B wants to block until A reads from the queue, so it
advertises that and sleeps. Waking up deadlock_timeout later, it notices that
A is waiting for B to write something. B will not spontaneously suspend
waiting to write to the queue, nor will A suspend waiting to read from the
queue. Thus, the deadlock is valid. This assumes the deadlock detector
reasons from an authoritative picture of pending waits and that we reliably
wake up a process when the condition it sought has arrived. We have that for
heavyweight lock acquisitions. The assumption may be incompatible with
Andres's hope, quoted above, of avoiding the full lock acquisition procedure.
I don't understand. What's typically going to happen here is that the
queue will initially be empty, and A will block. Suppose B has a
large message to write to the queue, let's say 100x the queue size.
It will write the first 1% of the message into the queue, set A's
latch, and go to sleep. A will now wake up, drain the queue, set B's
latch, and go to sleep. B will then wake up, write the next chunk of
the message, set A's latch again, and go to sleep. They'll go back
and forth like this until the entire message has been transmitted.
There's no deadlock here: everything is working as designed. But
there may be instants where both A and B are waiting, because (for
example) A may set B's latch and finish going to sleep before B gets
around to waking up.
That's probably something that we could patch around, but I think it's
missing the larger point. When you're dealing with locks, there is
one operation that causes blocking (acquiring a lock) and another
operation that unblocks other processes (releasing a lock). With
message queues, there are still two operations (reading and writing),
but either of them can either cause you to block yourself, or to
unblock another process. To put that another way, locks enforce
mutual exclusion; message queues force mutual *inclusion*. Locks
cause blocking when two processes try to operate on the same object at
the same time; message queues cause blocking *unless* two processes
operate on the same object at the same time. That difference seems to
me to be quite fundamental.
So then
you have to wonder why we're not solving problem #1, because the
deadlock was just as certain before we generated the maximum possible
number of tuples locally as it was afterwards.The "1." above reads like a benefit, not a problem. What might you solve
about it?
Sorry, that wasn't very clear. Normally, it is a benefit, but in some
cases, it could result in a long delay in reporting an inevitable
deadlock.
What I mean is: suppose we construct a working mechanism where the
deadlock detector knows about tuple queue waits. Suppose further that
we have a parallel leader and two workers cooperating to do a task
that takes 300 s and can be parallelized with perfect efficiency so
that, working together, those three processes can get the job done in
just 100 s. If the leader tries to take a lock held in a conflicting
mode by one of the workers, the worker will fill up the tuple queue -
probably quite quickly - and wait. We now detect a deadlock. Our
detection is timely, and life is good. On the other hand, suppose
that one of the *workers* tries to take a lock held in a conflicting
mode by the other worker or by the leader. There is no immediate
deadlock, because the leader is not waiting. Instead, we'll finish
the computation with the remaining worker and the leader, which will
take 150 seconds since we only have two processes instead of three,
and *then* the leader waits. I predict that users will be unhappy
about doing the whole computation - and only then detecting, well
after the time the query would normally have finished, a deadlock that
was inevitable from the beginning.
That problem is actually not too hard to avoid if you're OK with
extremely frequent lock manager manipulations. It's not a deadlock if
a worker is waiting for a lock (say, a relation extension lock) held
by the leader, because the leader might release that lock quickly.
But if the leader gets to the top of the funnel node's main loop and
one of its workers is still waiting for the lock at that point, that
is almost certainly a deadlock. Locks might get taken and released
during a single pass through that loop, but any lock held across
iterations is presumably going to be held until end-of-transaction, so
we're hosed. But checking for deadlock at the top of every main loop
iteration is far too expensive to contemplate. Even doing some much
lighter-weight manipulation of the lock manager data structures at the
top of every loop iteration is probably going to recreate the same
kind of lock manager contention problems that I tried to solve with
the fast-path locking stuff in 9.2.
To put all of my cards on the table, I've never really believed in
this approach. The reason we have a deadlock detector in the first
place is because we can't prevent users from making fundamentally
incompatible locking requests, like locking two tables in incompatible
orderings in two different sessions. But we *don't* have deadlock
detection for lwlocks because we (rightly) view it as our job to write
the code in such a way that deadlocks don't happen in the first place.
So we take locks in a predictable order, even in cases (like updating
a tuple) where that involves some fancy gymnastics. We could have
update always lock the old-tuple buffer before the new-tuple buffer
and make it the deadlock detector's job to sort that out, but that's
pretty obviously inferior. We can expect users to tolerate a deadlock
when it is their own actions that have precipitated the deadlock, but
it's a whole different thing to ask them to accept that what from the
user's point of view is an indivisible action (like an UPDATE, or a
parallel query, or perhaps an UPSERT) occasionally deadlocks for some
reason that they can neither understand nor prevent. Tuple queues are
an implementation detail. They shouldn't deadlock for the same reason
that lwlock acquisition shouldn't deadlock, and this whole project
should be entirely unnecessary.
The way we backed into worrying about exposing tuple queues to the
deadlock detector is that, if you let one backend in a parallel group
get and keep a lock on some resource and then another backend in the
same group tries to lock that resource and can't get the lock, you
will eventually an undetected deadlock. Andres's view is that we
should fix the deadlock detector to detect and report such cases, but
I think that's wrong. If a process in a parallel group can take and
retain until end of transaction a lock on some resource without which
other processes in the same parallel group cannot do useful work,
that's a BUG. We shouldn't need the deadlock detector to report that
problem for the same reason we shouldn't need the deadlock detector to
report lwlock-based deadlocks: if they are ever happening, you need to
fix your code, not the deadlock detector.
I think where this project went off the rails is when we made the
decision as to how to fix the problems in the original group locking
approach. In the original concept, locks between processes in the
same parallel group just didn't ever conflict; two
otherwise-conflicting locks held by different backends within the same
group were regarded as those processes sharing that lock. Andres and
possibly others pointed out that for stuff like relation locks, that's
probably not the right behavior. I agree with that. It was also
suggested that this would be less scary if we limited ourselves to
sharing the locks held at the beginning of parallelism. That had the
further advantage of making things like relation extension locks,
which won't be held at the starting of paralellism, unshared, while
relation locks would, in most cases, be shared. That felt pretty
good, so I did it, but I now think that was a mistake, because it
creates edge cases where parallel groups can self-deadlock. If we
instead regard relation locks between parallel group members as NEVER
conflicting, rather than conflicting only when both locks were
acquired after the start of parallelism, those edge cases go away.
The only downside is that if a worker A manages to do something to a
relation R that makes it unsafe for worker B to access, and worker B
then gets the lock anyway, we get no-holds-barred insanity instead of
a deadlock. But I am unconvinced that downside amounts to much,
because I believe the only way that A can make it unsafe for B to
access the relation is by doing something that CheckTableNotInUse()
will already prohibit in parallel mode. If it turns out there is some
oversight there, I don't think it'll be hard to plug. In contrast,
exposing this tuple queue wait information to the deadlock detector is
looking quite complex.
--
Robert Haas
EnterpriseDB: http://www.enterprisedb.com
The Enterprise PostgreSQL Company
--
Sent via pgsql-hackers mailing list (pgsql-hackers@postgresql.org)
To make changes to your subscription:
http://www.postgresql.org/mailpref/pgsql-hackers
On Tue, May 05, 2015 at 01:05:38PM -0400, Robert Haas wrote:
On Sat, May 2, 2015 at 12:35 PM, Noah Misch <noah@leadboat.com> wrote:
Perhaps, rather than model it as master M waiting on worker list W1|W2|W3,
model it with queue-nonempty and queue-nonfull events, one pair per queue.
That comment of mine was junk; it suggested a data structure containing the
same worker list it purported to remove. Oops.
Each individual queue has only a single reader and a single writer.
In your example, there would be three queues, not just one. Of
course, one could design a new shared memory data structure
representing a collection of related queues, but I still don't see
exactly how we get around the problem of that requiring
O(MaxBackends^2) storage space.
If each backend waits on a uniformly-distributed 50% of other backends,
tracking that wait graph indeed requires O(MaxBackends^2) space. Backends
completing useful work in the field will have far-sparser wait graphs, and
that should inform the choice of data structure:
On Tue, Apr 21, 2015 at 02:08:00PM -0400, Robert Haas wrote:
When I first started thinking about how to fix this, I said, well,
that's no big deal, we can just advertise the whole list of processes
that we're waiting for in shared memory, rather than just the one.
That boiled down to representing the wait graph with an adjacency list, which
sounds like an efficient choice.
M
subscribes to queue0-nonempty, and each of W1,W2,W3 publish to it. M
publishes to queue0-nonfull, and the workers subscribe. An edge forms in the
deadlock detector's graph when all of an event's subscribers wait on it. (One
can approximate that in userspace with a pair of advisory locks.)An edge between which two processes?
I hadn't de-fuzzed my thinking that far yet. If you still need the answer
after this message, let me know, and I'll work on that. As a guess, I think
it's three edges M-W1, M-W2 and M-W3.
After thinking about it a bit more, I realized that even if we settle
on some solution to that problem, there's another issues: the
wait-edges created by this system don't really have the same semantics
as regular lock waits. Suppose A is waiting on a lock held by B and B
is waiting for a lock held by A; that's a deadlock. But if A is
waiting for B to write to a tuple queue and B is waiting for A to read
from a tuple queue, that's not a deadlock if the queues in question
are the same.I do see a deadlock. B wants to block until A reads from the queue, so it
advertises that and sleeps. Waking up deadlock_timeout later, it notices that
A is waiting for B to write something. B will not spontaneously suspend
waiting to write to the queue, nor will A suspend waiting to read from the
queue. Thus, the deadlock is valid. This assumes the deadlock detector
reasons from an authoritative picture of pending waits and that we reliably
wake up a process when the condition it sought has arrived. We have that for
heavyweight lock acquisitions. The assumption may be incompatible with
Andres's hope, quoted above, of avoiding the full lock acquisition procedure.I don't understand. What's typically going to happen here is that the
queue will initially be empty, and A will block. Suppose B has a
large message to write to the queue, let's say 100x the queue size.
It will write the first 1% of the message into the queue, set A's
latch, and go to sleep. A will now wake up, drain the queue, set B's
latch, and go to sleep. B will then wake up, write the next chunk of
the message, set A's latch again, and go to sleep. They'll go back
and forth like this until the entire message has been transmitted.
There's no deadlock here: everything is working as designed. But
there may be instants where both A and B are waiting, because (for
example) A may set B's latch and finish going to sleep before B gets
around to waking up.
I see what you had in mind. The problem showed up in the last sentence;
namely, the hand-off from process to process is not atomic like it is for
heavyweight locks. That's exactly what I was (not too clearly) getting at
when I wrote, "This assumes ..." above. For the sake of illustration, suppose
you replace your queue in this algorithm with a heavyweight lock and a small
buffer. Initially, B holds the lock and A waits for the lock. The following
event sequence repeats until the transfer is done:
B fills the buffer
B releases the lock, granting it to A during LockRelease()
B starts waiting for the lock
A empties the buffer
A releases the lock, granting it to B during LockRelease()
A starts waiting for the lock
The deadlock detector will rightly never report a deadlock. To have the same
safety in your example, it's not enough for one process to set the latch of
another process. The process must update something in static shared memory to
indicate that the waited-for condition (queue-nonempty for A, queue-nonfull
for B) is now satisfied. You need such a unit of state anyway, don't you?
How else would the deadlock detector know that A waits for B to write to the
tuple queue? (When the deadlock detector notices this deadlock, it may be
running in a process not participating in the parallel group. It can't rely
on anything in backend or dynamic shared memory.) With that, the deadlock
detector can distinguish a process waiting for an yet-unsatisfied condition
from a process that will soon wake to exploit a recently-satisfied condition.
That's probably something that we could patch around, but I think it's
missing the larger point. When you're dealing with locks, there is
one operation that causes blocking (acquiring a lock) and another
operation that unblocks other processes (releasing a lock). With
message queues, there are still two operations (reading and writing),
but either of them can either cause you to block yourself, or to
unblock another process. To put that another way, locks enforce
mutual exclusion; message queues force mutual *inclusion*. Locks
cause blocking when two processes try to operate on the same object at
the same time; message queues cause blocking *unless* two processes
operate on the same object at the same time. That difference seems to
me to be quite fundamental.
Interesting thought.
That problem is actually not too hard to avoid if you're OK with
extremely frequent lock manager manipulations. It's not a deadlock if
a worker is waiting for a lock (say, a relation extension lock) held
by the leader, because the leader might release that lock quickly.
But if the leader gets to the top of the funnel node's main loop and
one of its workers is still waiting for the lock at that point, that
is almost certainly a deadlock. Locks might get taken and released
during a single pass through that loop, but any lock held across
iterations is presumably going to be held until end-of-transaction, so
we're hosed. But checking for deadlock at the top of every main loop
iteration is far too expensive to contemplate. Even doing some much
lighter-weight manipulation of the lock manager data structures at the
top of every loop iteration is probably going to recreate the same
kind of lock manager contention problems that I tried to solve with
the fast-path locking stuff in 9.2.
Agreed.
To put all of my cards on the table, I've never really believed in
this approach. The reason we have a deadlock detector in the first
place is because we can't prevent users from making fundamentally
incompatible locking requests, like locking two tables in incompatible
orderings in two different sessions. But we *don't* have deadlock
detection for lwlocks because we (rightly) view it as our job to write
the code in such a way that deadlocks don't happen in the first place.
So we take locks in a predictable order, even in cases (like updating
a tuple) where that involves some fancy gymnastics. We could have
update always lock the old-tuple buffer before the new-tuple buffer
and make it the deadlock detector's job to sort that out, but that's
pretty obviously inferior. We can expect users to tolerate a deadlock
when it is their own actions that have precipitated the deadlock, but
it's a whole different thing to ask them to accept that what from the
user's point of view is an indivisible action (like an UPDATE, or a
parallel query, or perhaps an UPSERT) occasionally deadlocks for some
reason that they can neither understand nor prevent. Tuple queues are
an implementation detail. They shouldn't deadlock for the same reason
that lwlock acquisition shouldn't deadlock, and this whole project
should be entirely unnecessary.The way we backed into worrying about exposing tuple queues to the
deadlock detector is that, if you let one backend in a parallel group
get and keep a lock on some resource and then another backend in the
same group tries to lock that resource and can't get the lock, you
will eventually an undetected deadlock. Andres's view is that we
should fix the deadlock detector to detect and report such cases, but
I think that's wrong. If a process in a parallel group can take and
retain until end of transaction a lock on some resource without which
other processes in the same parallel group cannot do useful work,
that's a BUG. We shouldn't need the deadlock detector to report that
problem for the same reason we shouldn't need the deadlock detector to
report lwlock-based deadlocks: if they are ever happening, you need to
fix your code, not the deadlock detector.
The number of held LWLocks is always zero when entering user-defined code and
again when exiting user-defined code. Therefore, the possible LWLock wait
graphs are known at compile time, and we could prove that none contain a
deadlock. Therefore, we scarcely miss having an LWLock deadlock detector.
That does not map to the tuple queue behavior at hand, because we hope to
allow the queue's writer to run user-defined code while the queue's reader is
waiting. My term "user-defined code" does come with some hand-waving. A
superuser can cause an undetected deadlock via a C-language hook. We would
not call that a PostgreSQL bug, though the hook is literally user-defined.
Let's keep the deadlock detector able to identify every deadlock a
non-superuser can cause. That includes deadlocks arising from heavyweight
lock acquisition in parallel-safe functions.
I think where this project went off the rails is when we made the
decision as to how to fix the problems in the original group locking
approach. In the original concept, locks between processes in the
same parallel group just didn't ever conflict; two
otherwise-conflicting locks held by different backends within the same
group were regarded as those processes sharing that lock. Andres and
possibly others pointed out that for stuff like relation locks, that's
I think you mean "relation extension locks".
probably not the right behavior. I agree with that. It was also
suggested that this would be less scary if we limited ourselves to
sharing the locks held at the beginning of parallelism. That had the
further advantage of making things like relation extension locks,
which won't be held at the starting of paralellism, unshared, while
relation locks would, in most cases, be shared. That felt pretty
good, so I did it, but I now think that was a mistake, because it
creates edge cases where parallel groups can self-deadlock. If we
instead regard relation locks between parallel group members as NEVER
conflicting, rather than conflicting only when both locks were
acquired after the start of parallelism, those edge cases go away.
Yes. Then you're back to something like the LWLock scenario, where an
undetected deadlock implies a PostgreSQL bug. That's a good place to be.
The only downside is that if a worker A manages to do something to a
relation R that makes it unsafe for worker B to access, and worker B
then gets the lock anyway, we get no-holds-barred insanity instead of
a deadlock. But I am unconvinced that downside amounts to much,
because I believe the only way that A can make it unsafe for B to
access the relation is by doing something that CheckTableNotInUse()
will already prohibit in parallel mode. If it turns out there is some
oversight there, I don't think it'll be hard to plug.
This is the bottom line, and I agree. I wrote more about that here:
/messages/by-id/20141226040546.GC1971688@tornado.leadboat.com
I admit that it's alarming to have different conflict semantics by locktag.
The tension originates, I think, from e.g. LOCKTAG_RELATION_EXTEND serving two
distinct purposes simultaneously. As I wrote in the mail just cited, before
altering a table in a manner that threatens concurrent usage, one must ensure
that (a) other transactions and (b) other parts of my own transaction aren't
in the way. Customarily, a heavyweight lock rules out (a), and
CheckTableNotInUse() rules out (b). Relation extension does not have or need
a CheckTableNotInUse() call or similar, because it doesn't call arbitrary code
that might reenter the relation extension process. Under parallelism, though,
it will be possible for multiple processes of a given transaction to attempt
relation extension simultaneously. So we need a (b) test. It just so happens
that allowing LOCKTAG_RELATION_EXTEND to conflict in a parallel group causes
that lock acquisition to suffice for both purposes (a) and (b). That's neat
but arguably impure. Every cure I've pondered has been worse than the
disease, though. It will be okay.
In contrast,
exposing this tuple queue wait information to the deadlock detector is
looking quite complex.
Yep.
Thanks,
nm
--
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, May 7, 2015 at 3:09 AM, Noah Misch <noah@leadboat.com> wrote:
Each individual queue has only a single reader and a single writer.
In your example, there would be three queues, not just one. Of
course, one could design a new shared memory data structure
representing a collection of related queues, but I still don't see
exactly how we get around the problem of that requiring
O(MaxBackends^2) storage space.If each backend waits on a uniformly-distributed 50% of other backends,
tracking that wait graph indeed requires O(MaxBackends^2) space. Backends
completing useful work in the field will have far-sparser wait graphs, and
that should inform the choice of data structure:
We can, of course, underallocate and hope for the best.
M
subscribes to queue0-nonempty, and each of W1,W2,W3 publish to it. M
publishes to queue0-nonfull, and the workers subscribe. An edge forms in the
deadlock detector's graph when all of an event's subscribers wait on it. (One
can approximate that in userspace with a pair of advisory locks.)An edge between which two processes?
I hadn't de-fuzzed my thinking that far yet. If you still need the answer
after this message, let me know, and I'll work on that. As a guess, I think
it's three edges M-W1, M-W2 and M-W3.
I think it's an edge where one end is M and the other end is fuzzily
diffused across W1, W2, and W3, because we only need one of them to
wake up. There's no way to represent such a thing in the deadlock
detector today. We could invent one, of course, but it sounds
complicated. Also, I suspect deadlock checking in this world reduces
to http://en.wikipedia.org/wiki/Boolean_satisfiability_problem and is
therefore NP-complete.
I see what you had in mind. The problem showed up in the last sentence;
namely, the hand-off from process to process is not atomic like it is for
heavyweight locks. That's exactly what I was (not too clearly) getting at
when I wrote, "This assumes ..." above. For the sake of illustration, suppose
you replace your queue in this algorithm with a heavyweight lock and a small
buffer. Initially, B holds the lock and A waits for the lock. The following
event sequence repeats until the transfer is done:B fills the buffer
B releases the lock, granting it to A during LockRelease()
B starts waiting for the lock
A empties the buffer
A releases the lock, granting it to B during LockRelease()
A starts waiting for the lockThe deadlock detector will rightly never report a deadlock. To have the same
safety in your example, it's not enough for one process to set the latch of
another process. The process must update something in static shared memory to
indicate that the waited-for condition (queue-nonempty for A, queue-nonfull
for B) is now satisfied. You need such a unit of state anyway, don't you?
Yes. It's the number of bytes read and written, and it's stored in
dynamic shared memory. You can move some of the state to the main
shared memory segment, but I don't look forward to the performance
consequences of a lock acquire & release cycle every time the queue
fills or drains. Of course, there is also a loss of flexibility
there: a big reason for developing this stuff in the first place was
to avoid being constrained by the main shared memory segment.
The number of held LWLocks is always zero when entering user-defined code and
again when exiting user-defined code. Therefore, the possible LWLock wait
graphs are known at compile time, and we could prove that none contain a
deadlock. Therefore, we scarcely miss having an LWLock deadlock detector.
That does not map to the tuple queue behavior at hand, because we hope to
allow the queue's writer to run user-defined code while the queue's reader is
waiting. My term "user-defined code" does come with some hand-waving. A
superuser can cause an undetected deadlock via a C-language hook. We would
not call that a PostgreSQL bug, though the hook is literally user-defined.
Let's keep the deadlock detector able to identify every deadlock a
non-superuser can cause. That includes deadlocks arising from heavyweight
lock acquisition in parallel-safe functions.
I'm in agreement with that goal.
I think where this project went off the rails is when we made the
decision as to how to fix the problems in the original group locking
approach. In the original concept, locks between processes in the
same parallel group just didn't ever conflict; two
otherwise-conflicting locks held by different backends within the same
group were regarded as those processes sharing that lock. Andres and
possibly others pointed out that for stuff like relation locks, that'sI think you mean "relation extension locks".
Yes, thanks.
probably not the right behavior. I agree with that. It was also
suggested that this would be less scary if we limited ourselves to
sharing the locks held at the beginning of parallelism. That had the
further advantage of making things like relation extension locks,
which won't be held at the starting of paralellism, unshared, while
relation locks would, in most cases, be shared. That felt pretty
good, so I did it, but I now think that was a mistake, because it
creates edge cases where parallel groups can self-deadlock. If we
instead regard relation locks between parallel group members as NEVER
conflicting, rather than conflicting only when both locks were
acquired after the start of parallelism, those edge cases go away.Yes. Then you're back to something like the LWLock scenario, where an
undetected deadlock implies a PostgreSQL bug. That's a good place to be.
Good.
The only downside is that if a worker A manages to do something to a
relation R that makes it unsafe for worker B to access, and worker B
then gets the lock anyway, we get no-holds-barred insanity instead of
a deadlock. But I am unconvinced that downside amounts to much,
because I believe the only way that A can make it unsafe for B to
access the relation is by doing something that CheckTableNotInUse()
will already prohibit in parallel mode. If it turns out there is some
oversight there, I don't think it'll be hard to plug.This is the bottom line, and I agree. I wrote more about that here:
/messages/by-id/20141226040546.GC1971688@tornado.leadboat.comI admit that it's alarming to have different conflict semantics by locktag.
The tension originates, I think, from e.g. LOCKTAG_RELATION_EXTEND serving two
distinct purposes simultaneously. As I wrote in the mail just cited, before
altering a table in a manner that threatens concurrent usage, one must ensure
that (a) other transactions and (b) other parts of my own transaction aren't
in the way. Customarily, a heavyweight lock rules out (a), and
CheckTableNotInUse() rules out (b). Relation extension does not have or need
a CheckTableNotInUse() call or similar, because it doesn't call arbitrary code
that might reenter the relation extension process. Under parallelism, though,
it will be possible for multiple processes of a given transaction to attempt
relation extension simultaneously. So we need a (b) test. It just so happens
that allowing LOCKTAG_RELATION_EXTEND to conflict in a parallel group causes
that lock acquisition to suffice for both purposes (a) and (b). That's neat
but arguably impure. Every cure I've pondered has been worse than the
disease, though. It will be okay.
That's my opinion as 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