diff --git a/src/backend/access/transam/README.parallel b/src/backend/access/transam/README.parallel
index 4d98f4e65d18b4c09d7bf214180b0628642f2d09..5c33c40ae957080cef3c94ed4145556016436f11 100644
--- a/src/backend/access/transam/README.parallel
+++ b/src/backend/access/transam/README.parallel
@@ -198,7 +198,7 @@ pattern looks like this:
 
 	EnterParallelMode();		/* prohibit unsafe state changes */
 
-	pcxt = CreateParallelContext(entrypoint, nworkers);
+	pcxt = CreateParallelContext("library_name", "function_name", nworkers);
 
 	/* Allow space for application-specific data here. */
 	shm_toc_estimate_chunk(&pcxt->estimator, size);
diff --git a/src/backend/access/transam/parallel.c b/src/backend/access/transam/parallel.c
index b3d3853fbc202662bb9c369900a61f6e790a2013..04fa2ed455026eb02f73562daafb0c1a14f59595 100644
--- a/src/backend/access/transam/parallel.c
+++ b/src/backend/access/transam/parallel.c
@@ -19,6 +19,7 @@
 #include "access/xlog.h"
 #include "catalog/namespace.h"
 #include "commands/async.h"
+#include "executor/execParallel.h"
 #include "libpq/libpq.h"
 #include "libpq/pqformat.h"
 #include "libpq/pqmq.h"
@@ -61,7 +62,7 @@
 #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_ENTRYPOINT				UINT64CONST(0xFFFFFFFFFFFF0009)
 
 /* Fixed-size parallel state. */
 typedef struct FixedParallelState
@@ -77,9 +78,6 @@ typedef struct FixedParallelState
 	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;
 
@@ -107,10 +105,26 @@ static FixedParallelState *MyFixedParallelState;
 /* List of active parallel contexts. */
 static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list);
 
+/*
+ * List of internal parallel worker entry points.  We need this for
+ * reasons explained in LookupParallelWorkerFunction(), below.
+ */
+static const struct
+{
+	const char *fn_name;
+	parallel_worker_main_type fn_addr;
+}	InternalParallelWorkers[] =
+
+{
+	{
+		"ParallelQueryMain", ParallelQueryMain
+	}
+};
+
 /* Private functions. */
 static void HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg);
-static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
 static void WaitForParallelWorkersToExit(ParallelContext *pcxt);
+static parallel_worker_main_type LookupParallelWorkerFunction(const char *libraryname, const char *funcname);
 
 
 /*
@@ -119,7 +133,8 @@ static void WaitForParallelWorkersToExit(ParallelContext *pcxt);
  * destroyed before exiting the current subtransaction.
  */
 ParallelContext *
-CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
+CreateParallelContext(const char *library_name, const char *function_name,
+					  int nworkers)
 {
 	MemoryContext oldcontext;
 	ParallelContext *pcxt;
@@ -152,7 +167,8 @@ CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
 	pcxt = palloc0(sizeof(ParallelContext));
 	pcxt->subid = GetCurrentSubTransactionId();
 	pcxt->nworkers = nworkers;
-	pcxt->entrypoint = entrypoint;
+	pcxt->library_name = pstrdup(library_name);
+	pcxt->function_name = pstrdup(function_name);
 	pcxt->error_context_stack = error_context_stack;
 	shm_toc_initialize_estimator(&pcxt->estimator);
 	dlist_push_head(&pcxt_list, &pcxt->node);
@@ -163,33 +179,6 @@ CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
 	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
  * copy state and other bookkeeping information that will be needed by
@@ -249,15 +238,10 @@ InitializeParallelDSM(ParallelContext *pcxt)
 										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);
-		}
+		/* Estimate how much we'll need for the entrypoint info. */
+		shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name) +
+							   strlen(pcxt->function_name) + 2);
+		shm_toc_estimate_keys(&pcxt->estimator, 1);
 	}
 
 	/*
@@ -297,7 +281,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
 	fps->parallel_master_pgproc = MyProc;
 	fps->parallel_master_pid = MyProcPid;
 	fps->parallel_master_backend_id = MyBackendId;
-	fps->entrypoint = pcxt->entrypoint;
 	SpinLockInit(&fps->mutex);
 	fps->last_xlog_end = 0;
 	shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
@@ -312,6 +295,8 @@ InitializeParallelDSM(ParallelContext *pcxt)
 		char	   *asnapspace;
 		char	   *tstatespace;
 		char	   *error_queue_space;
+		char	   *entrypointstate;
+		Size		lnamelen;
 
 		/* Serialize shared libraries we have loaded. */
 		libraryspace = shm_toc_allocate(pcxt->toc, library_len);
@@ -368,19 +353,19 @@ InitializeParallelDSM(ParallelContext *pcxt)
 		}
 		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);
-		}
+		/*
+		 * Serialize entrypoint information.  It's unsafe to pass function
+		 * pointers across processes, as the function pointer may be different
+		 * in each process in EXEC_BACKEND builds, so we always pass library
+		 * and function name.  (We use library name "postgres" for functions
+		 * in the core backend.)
+		 */
+		lnamelen = strlen(pcxt->library_name);
+		entrypointstate = shm_toc_allocate(pcxt->toc, lnamelen +
+										   strlen(pcxt->function_name) + 2);
+		strcpy(entrypointstate, pcxt->library_name);
+		strcpy(entrypointstate + lnamelen + 1, pcxt->function_name);
+		shm_toc_insert(pcxt->toc, PARALLEL_KEY_ENTRYPOINT, entrypointstate);
 	}
 
 	/* Restore previous memory context. */
@@ -671,6 +656,8 @@ DestroyParallelContext(ParallelContext *pcxt)
 	}
 
 	/* Free memory. */
+	pfree(pcxt->library_name);
+	pfree(pcxt->function_name);
 	pfree(pcxt);
 }
 
@@ -941,6 +928,10 @@ ParallelWorkerMain(Datum main_arg)
 	shm_mq	   *mq;
 	shm_mq_handle *mqh;
 	char	   *libraryspace;
+	char	   *entrypointstate;
+	char	   *library_name;
+	char	   *function_name;
+	parallel_worker_main_type entrypt;
 	char	   *gucspace;
 	char	   *combocidspace;
 	char	   *tsnapspace;
@@ -1040,6 +1031,18 @@ ParallelWorkerMain(Datum main_arg)
 	Assert(libraryspace != NULL);
 	RestoreLibraryState(libraryspace);
 
+	/*
+	 * Identify the entry point to be called.  In theory this could result in
+	 * loading an additional library, though most likely the entry point is in
+	 * the core backend or in a library we just loaded.
+	 */
+	entrypointstate = shm_toc_lookup(toc, PARALLEL_KEY_ENTRYPOINT);
+	Assert(entrypointstate != NULL);
+	library_name = entrypointstate;
+	function_name = entrypointstate + strlen(library_name) + 1;
+
+	entrypt = LookupParallelWorkerFunction(library_name, function_name);
+
 	/* Restore database connection. */
 	BackgroundWorkerInitializeConnectionByOid(fps->database_id,
 											  fps->authenticated_user_id);
@@ -1102,11 +1105,8 @@ ParallelWorkerMain(Datum main_arg)
 
 	/*
 	 * 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);
+	entrypt(seg, toc);
 
 	/* Must exit parallel mode to pop active snapshot. */
 	ExitParallelMode();
@@ -1121,33 +1121,6 @@ ParallelWorkerMain(Datum main_arg)
 	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);
-}
-
 /*
  * Update shared memory with the ending location of the last WAL record we
  * wrote, if it's greater than the value already stored there.
@@ -1163,3 +1136,47 @@ ParallelWorkerReportLastRecEnd(XLogRecPtr last_xlog_end)
 		fps->last_xlog_end = last_xlog_end;
 	SpinLockRelease(&fps->mutex);
 }
+
+/*
+ * Look up (and possibly load) a parallel worker entry point function.
+ *
+ * For functions contained in the core code, we use library name "postgres"
+ * and consult the InternalParallelWorkers array.  External functions are
+ * looked up, and loaded if necessary, using load_external_function().
+ *
+ * The point of this is to pass function names as strings across process
+ * boundaries.  We can't pass actual function addresses because of the
+ * possibility that the function has been loaded at a different address
+ * in a different process.  This is obviously a hazard for functions in
+ * loadable libraries, but it can happen even for functions in the core code
+ * on platforms using EXEC_BACKEND (e.g., Windows).
+ *
+ * At some point it might be worthwhile to get rid of InternalParallelWorkers[]
+ * in favor of applying load_external_function() for core functions too;
+ * but that raises portability issues that are not worth addressing now.
+ */
+static parallel_worker_main_type
+LookupParallelWorkerFunction(const char *libraryname, const char *funcname)
+{
+	/*
+	 * If the function is to be loaded from postgres itself, search the
+	 * InternalParallelWorkers array.
+	 */
+	if (strcmp(libraryname, "postgres") == 0)
+	{
+		int			i;
+
+		for (i = 0; i < lengthof(InternalParallelWorkers); i++)
+		{
+			if (strcmp(InternalParallelWorkers[i].fn_name, funcname) == 0)
+				return InternalParallelWorkers[i].fn_addr;
+		}
+
+		/* We can only reach this by programming error. */
+		elog(ERROR, "internal function \"%s\" not found", funcname);
+	}
+
+	/* Otherwise load from external library. */
+	return (parallel_worker_main_type)
+		load_external_function(libraryname, funcname, true, NULL);
+}
diff --git a/src/backend/executor/execParallel.c b/src/backend/executor/execParallel.c
index 5fd4ee0b8249e9deda344165c7252c761d3e8a32..9c98f5492e81b9687b812a3021faf1c860850ee1 100644
--- a/src/backend/executor/execParallel.c
+++ b/src/backend/executor/execParallel.c
@@ -112,8 +112,7 @@ static shm_mq_handle **ExecParallelSetupTupleQueues(ParallelContext *pcxt,
 static bool ExecParallelRetrieveInstrumentation(PlanState *planstate,
 							 SharedExecutorInstrumentation *instrumentation);
 
-/* Helper functions that run in the parallel worker. */
-static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+/* Helper function that runs in the parallel worker. */
 static DestReceiver *ExecParallelGetReceiver(dsm_segment *seg, shm_toc *toc);
 
 /*
@@ -410,7 +409,7 @@ ExecInitParallelPlan(PlanState *planstate, EState *estate, int nworkers)
 	pstmt_data = ExecSerializePlan(planstate->plan, estate);
 
 	/* Create a parallel context. */
-	pcxt = CreateParallelContext(ParallelQueryMain, nworkers);
+	pcxt = CreateParallelContext("postgres", "ParallelQueryMain", nworkers);
 	pei->pcxt = pcxt;
 
 	/*
@@ -831,7 +830,7 @@ ExecParallelInitializeWorker(PlanState *planstate, shm_toc *toc)
  * to do this are also stored in the dsm_segment and can be accessed through
  * the shm_toc.
  */
-static void
+void
 ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
 {
 	BufferUsage *buffer_usage;
diff --git a/src/backend/postmaster/bgworker.c b/src/backend/postmaster/bgworker.c
index 9ad3e915db30efb5645be3f315fb585612a6a04d..f1194891f5091f7735a2e3f44685d096b2776cf2 100644
--- a/src/backend/postmaster/bgworker.c
+++ b/src/backend/postmaster/bgworker.c
@@ -111,25 +111,30 @@ struct BackgroundWorkerHandle
 static BackgroundWorkerArray *BackgroundWorkerData;
 
 /*
- * List of internal background workers. These are used for mapping the
- * function name to actual function when building with EXEC_BACKEND and also
- * to allow these to be loaded outside of shared_preload_libraries.
+ * List of internal background worker entry points.  We need this for
+ * reasons explained in LookupBackgroundWorkerFunction(), below.
  */
-typedef struct InternalBGWorkerMain
+static const struct
 {
-	char			   *bgw_function_name;
-	bgworker_main_type	bgw_main;
-} InternalBGWorkerMain;
-
-static const InternalBGWorkerMain InternalBGWorkers[] = {
-	{"ParallelWorkerMain", ParallelWorkerMain},
-	{"ApplyLauncherMain", ApplyLauncherMain},
-	{"ApplyWorkerMain", ApplyWorkerMain},
-	/* Dummy entry marking end of the array. */
-	{NULL, NULL}
+	const char *fn_name;
+	bgworker_main_type fn_addr;
+}	InternalBGWorkers[] =
+
+{
+	{
+		"ParallelWorkerMain", ParallelWorkerMain
+	},
+	{
+		"ApplyLauncherMain", ApplyLauncherMain
+	},
+	{
+		"ApplyWorkerMain", ApplyWorkerMain
+	}
 };
 
-static bgworker_main_type GetInternalBgWorkerMain(BackgroundWorker *worker);
+/* Private functions. */
+static bgworker_main_type LookupBackgroundWorkerFunction(const char *libraryname, const char *funcname);
+
 
 /*
  * Calculate shared memory needed.
@@ -812,18 +817,10 @@ StartBackgroundWorker(void)
 	}
 
 	/*
-	 * For internal workers set the entry point to known function address.
-	 * Otherwise use the entry point specified by library name (which will
-	 * be loaded, if necessary) and a function name (which will be looked up
-	 * in the named library).
+	 * Look up the entry point function, loading its library if necessary.
 	 */
-	entrypt = GetInternalBgWorkerMain(worker);
-
-	if (entrypt == NULL)
-		entrypt = (bgworker_main_type)
-			load_external_function(worker->bgw_library_name,
-								   worker->bgw_function_name,
-								   true, NULL);
+	entrypt = LookupBackgroundWorkerFunction(worker->bgw_library_name,
+											 worker->bgw_function_name);
 
 	/*
 	 * Note that in normal processes, we would call InitPostgres here.  For a
@@ -842,10 +839,11 @@ StartBackgroundWorker(void)
 }
 
 /*
- * Register a new background worker while processing shared_preload_libraries.
+ * Register a new static background worker.
  *
- * This can only be called in the _PG_init function of a module library
- * that's loaded by shared_preload_libraries; otherwise it has no effect.
+ * This can only be called directly from postmaster or in the _PG_init
+ * function of a module library that's loaded by shared_preload_libraries;
+ * otherwise it will have no effect.
  */
 void
 RegisterBackgroundWorker(BackgroundWorker *worker)
@@ -858,7 +856,7 @@ RegisterBackgroundWorker(BackgroundWorker *worker)
 		 (errmsg("registering background worker \"%s\"", worker->bgw_name)));
 
 	if (!process_shared_preload_libraries_in_progress &&
-		GetInternalBgWorkerMain(worker) == NULL)
+		strcmp(worker->bgw_library_name, "postgres") != 0)
 	{
 		if (!IsUnderPostmaster)
 			ereport(LOG,
@@ -1193,26 +1191,45 @@ TerminateBackgroundWorker(BackgroundWorkerHandle *handle)
 }
 
 /*
- * Search the known internal worker array and return its main function
- * pointer if found.
+ * Look up (and possibly load) a bgworker entry point function.
+ *
+ * For functions contained in the core code, we use library name "postgres"
+ * and consult the InternalBGWorkers array.  External functions are
+ * looked up, and loaded if necessary, using load_external_function().
  *
- * Returns NULL if not known internal worker.
+ * The point of this is to pass function names as strings across process
+ * boundaries.  We can't pass actual function addresses because of the
+ * possibility that the function has been loaded at a different address
+ * in a different process.  This is obviously a hazard for functions in
+ * loadable libraries, but it can happen even for functions in the core code
+ * on platforms using EXEC_BACKEND (e.g., Windows).
+ *
+ * At some point it might be worthwhile to get rid of InternalBGWorkers[]
+ * in favor of applying load_external_function() for core functions too;
+ * but that raises portability issues that are not worth addressing now.
  */
 static bgworker_main_type
-GetInternalBgWorkerMain(BackgroundWorker *worker)
+LookupBackgroundWorkerFunction(const char *libraryname, const char *funcname)
 {
-	int i;
+	/*
+	 * If the function is to be loaded from postgres itself, search the
+	 * InternalBGWorkers array.
+	 */
+	if (strcmp(libraryname, "postgres") == 0)
+	{
+		int			i;
 
-	/* Internal workers always have to use postgres as library name. */
-	if (strncmp(worker->bgw_library_name, "postgres", BGW_MAXLEN) != 0)
-		return NULL;
+		for (i = 0; i < lengthof(InternalBGWorkers); i++)
+		{
+			if (strcmp(InternalBGWorkers[i].fn_name, funcname) == 0)
+				return InternalBGWorkers[i].fn_addr;
+		}
 
-	for (i = 0; InternalBGWorkers[i].bgw_function_name; i++)
-	{
-		if (strncmp(InternalBGWorkers[i].bgw_function_name,
-					worker->bgw_function_name, BGW_MAXLEN) == 0)
-			return InternalBGWorkers[i].bgw_main;
+		/* We can only reach this by programming error. */
+		elog(ERROR, "internal function \"%s\" not found", funcname);
 	}
 
-	return NULL;
+	/* Otherwise load from external library. */
+	return (bgworker_main_type)
+		load_external_function(libraryname, funcname, true, NULL);
 }
diff --git a/src/backend/utils/fmgr/dfmgr.c b/src/backend/utils/fmgr/dfmgr.c
index 94d7dac94ce3e803653ee1f5590bf2ecfa716c6b..9739c4c14470d473b29760b4f9632fd7141fe05d 100644
--- a/src/backend/utils/fmgr/dfmgr.c
+++ b/src/backend/utils/fmgr/dfmgr.c
@@ -91,7 +91,7 @@ static const Pg_magic_struct magic_data = PG_MODULE_MAGIC_DATA;
  * at less cost than repeating load_external_function.
  */
 PGFunction
-load_external_function(char *filename, char *funcname,
+load_external_function(const char *filename, const char *funcname,
 					   bool signalNotFound, void **filehandle)
 {
 	char	   *fullname;
@@ -108,8 +108,12 @@ load_external_function(char *filename, char *funcname,
 	if (filehandle)
 		*filehandle = lib_handle;
 
-	/* Look up the function within the library */
-	retval = (PGFunction) pg_dlsym(lib_handle, funcname);
+	/*
+	 * Look up the function within the library.  According to POSIX dlsym()
+	 * should declare its second argument as "const char *", but older
+	 * platforms might not, so for the time being we just cast away const.
+	 */
+	retval = (PGFunction) pg_dlsym(lib_handle, (char *) funcname);
 
 	if (retval == NULL && signalNotFound)
 		ereport(ERROR,
@@ -155,9 +159,10 @@ load_file(const char *filename, bool restricted)
  * Return (PGFunction) NULL if not found.
  */
 PGFunction
-lookup_external_function(void *filehandle, char *funcname)
+lookup_external_function(void *filehandle, const char *funcname)
 {
-	return (PGFunction) pg_dlsym(filehandle, funcname);
+	/* as above, cast away const for the time being */
+	return (PGFunction) pg_dlsym(filehandle, (char *) funcname);
 }
 
 
diff --git a/src/backend/utils/fmgr/fmgr.c b/src/backend/utils/fmgr/fmgr.c
index 68d2110890aadea93caa86d1172fb16d52e6999d..d9e3bf240db42bfea51bda3786d2e5b4e1336f22 100644
--- a/src/backend/utils/fmgr/fmgr.c
+++ b/src/backend/utils/fmgr/fmgr.c
@@ -381,7 +381,7 @@ fmgr_info_other_lang(Oid functionId, FmgrInfo *finfo, HeapTuple procedureTuple)
  * pg_proc.
  */
 const Pg_finfo_record *
-fetch_finfo_record(void *filehandle, char *funcname)
+fetch_finfo_record(void *filehandle, const char *funcname)
 {
 	char	   *infofuncname;
 	PGFInfoFunction infofunc;
diff --git a/src/include/access/parallel.h b/src/include/access/parallel.h
index 5065a3830cf5700d7c0617b7b88936b0809748ab..590e27a48455466ad6d1aacf59e290c7f7e1de7b 100644
--- a/src/include/access/parallel.h
+++ b/src/include/access/parallel.h
@@ -35,7 +35,6 @@ typedef struct ParallelContext
 	SubTransactionId subid;
 	int			nworkers;
 	int			nworkers_launched;
-	parallel_worker_main_type entrypoint;
 	char	   *library_name;
 	char	   *function_name;
 	ErrorContextCallback *error_context_stack;
@@ -52,8 +51,7 @@ extern bool InitializingParallelWorker;
 
 #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 ParallelContext *CreateParallelContext(const char *library_name, const char *function_name, int nworkers);
 extern void InitializeParallelDSM(ParallelContext *pcxt);
 extern void ReinitializeParallelDSM(ParallelContext *pcxt);
 extern void LaunchParallelWorkers(ParallelContext *pcxt);
diff --git a/src/include/executor/execParallel.h b/src/include/executor/execParallel.h
index 8bc4270e1771d923e2b5ccf4d43744943cc4f99a..0b7ca59dca566c827d7a18b9e7dd853ad1be3725 100644
--- a/src/include/executor/execParallel.h
+++ b/src/include/executor/execParallel.h
@@ -38,4 +38,6 @@ extern void ExecParallelFinish(ParallelExecutorInfo *pei);
 extern void ExecParallelCleanup(ParallelExecutorInfo *pei);
 extern void ExecParallelReinitialize(ParallelExecutorInfo *pei);
 
+extern void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
+
 #endif   /* EXECPARALLEL_H */
diff --git a/src/include/fmgr.h b/src/include/fmgr.h
index 0c695e246a5f70ff99afd88a8a16d125a64e6acd..cfb7b7774df70077bd6d1ea8acb4bdd6f424fda2 100644
--- a/src/include/fmgr.h
+++ b/src/include/fmgr.h
@@ -657,7 +657,7 @@ extern bytea *OidSendFunctionCall(Oid functionId, Datum val);
 /*
  * Routines in fmgr.c
  */
-extern const Pg_finfo_record *fetch_finfo_record(void *filehandle, char *funcname);
+extern const Pg_finfo_record *fetch_finfo_record(void *filehandle, const char *funcname);
 extern void clear_external_function_hash(void *filehandle);
 extern Oid	fmgr_internal_function(const char *proname);
 extern Oid	get_fn_expr_rettype(FmgrInfo *flinfo);
@@ -673,9 +673,9 @@ extern bool CheckFunctionValidatorAccess(Oid validatorOid, Oid functionOid);
  */
 extern char *Dynamic_library_path;
 
-extern PGFunction load_external_function(char *filename, char *funcname,
+extern PGFunction load_external_function(const char *filename, const char *funcname,
 					   bool signalNotFound, void **filehandle);
-extern PGFunction lookup_external_function(void *filehandle, char *funcname);
+extern PGFunction lookup_external_function(void *filehandle, const char *funcname);
 extern void load_file(const char *filename, bool restricted);
 extern void **find_rendezvous_variable(const char *varName);
 extern Size EstimateLibraryStateSpace(void);