Avoid passing function pointers across process boundaries.

We'd already recognized that we can't pass function pointers across process
boundaries for functions in loadable modules, since a shared library could
get loaded at different addresses in different processes.  But actually the
practice doesn't work for functions in the core backend either, if we're
using EXEC_BACKEND.  This is the cause of recent failures on buildfarm
member culicidae.  Switch to passing a string function name in all cases.

Something like this needs to be back-patched into 9.6, but let's see
if the buildfarm likes it first.

Petr Jelinek, with a bunch of basically-cosmetic adjustments by me

Discussion: https://postgr.es/m/548f9c1d-eafa-e3fa-9da8-f0cc2f654e60@2ndquadrant.com
This commit is contained in:
Tom Lane 2017-04-14 23:50:16 -04:00
parent 5a617ab3e6
commit 32470825d3
9 changed files with 187 additions and 149 deletions

View File

@ -198,7 +198,7 @@ pattern looks like this:
EnterParallelMode(); /* prohibit unsafe state changes */ EnterParallelMode(); /* prohibit unsafe state changes */
pcxt = CreateParallelContext(entrypoint, nworkers); pcxt = CreateParallelContext("library_name", "function_name", nworkers);
/* Allow space for application-specific data here. */ /* Allow space for application-specific data here. */
shm_toc_estimate_chunk(&pcxt->estimator, size); shm_toc_estimate_chunk(&pcxt->estimator, size);

View File

@ -19,6 +19,7 @@
#include "access/xlog.h" #include "access/xlog.h"
#include "catalog/namespace.h" #include "catalog/namespace.h"
#include "commands/async.h" #include "commands/async.h"
#include "executor/execParallel.h"
#include "libpq/libpq.h" #include "libpq/libpq.h"
#include "libpq/pqformat.h" #include "libpq/pqformat.h"
#include "libpq/pqmq.h" #include "libpq/pqmq.h"
@ -61,7 +62,7 @@
#define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006) #define PARALLEL_KEY_TRANSACTION_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0006)
#define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007) #define PARALLEL_KEY_ACTIVE_SNAPSHOT UINT64CONST(0xFFFFFFFFFFFF0007)
#define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008) #define PARALLEL_KEY_TRANSACTION_STATE UINT64CONST(0xFFFFFFFFFFFF0008)
#define PARALLEL_KEY_EXTENSION_TRAMPOLINE UINT64CONST(0xFFFFFFFFFFFF0009) #define PARALLEL_KEY_ENTRYPOINT UINT64CONST(0xFFFFFFFFFFFF0009)
/* Fixed-size parallel state. */ /* Fixed-size parallel state. */
typedef struct FixedParallelState typedef struct FixedParallelState
@ -77,9 +78,6 @@ typedef struct FixedParallelState
pid_t parallel_master_pid; pid_t parallel_master_pid;
BackendId parallel_master_backend_id; BackendId parallel_master_backend_id;
/* Entrypoint for parallel workers. */
parallel_worker_main_type entrypoint;
/* Mutex protects remaining fields. */ /* Mutex protects remaining fields. */
slock_t mutex; slock_t mutex;
@ -107,10 +105,26 @@ static FixedParallelState *MyFixedParallelState;
/* List of active parallel contexts. */ /* List of active parallel contexts. */
static dlist_head pcxt_list = DLIST_STATIC_INIT(pcxt_list); 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. */ /* Private functions. */
static void HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg); static void HandleParallelMessage(ParallelContext *pcxt, int i, StringInfo msg);
static void ParallelExtensionTrampoline(dsm_segment *seg, shm_toc *toc);
static void WaitForParallelWorkersToExit(ParallelContext *pcxt); 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. * destroyed before exiting the current subtransaction.
*/ */
ParallelContext * ParallelContext *
CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers) CreateParallelContext(const char *library_name, const char *function_name,
int nworkers)
{ {
MemoryContext oldcontext; MemoryContext oldcontext;
ParallelContext *pcxt; ParallelContext *pcxt;
@ -152,7 +167,8 @@ CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
pcxt = palloc0(sizeof(ParallelContext)); pcxt = palloc0(sizeof(ParallelContext));
pcxt->subid = GetCurrentSubTransactionId(); pcxt->subid = GetCurrentSubTransactionId();
pcxt->nworkers = nworkers; 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; pcxt->error_context_stack = error_context_stack;
shm_toc_initialize_estimator(&pcxt->estimator); shm_toc_initialize_estimator(&pcxt->estimator);
dlist_push_head(&pcxt_list, &pcxt->node); dlist_push_head(&pcxt_list, &pcxt->node);
@ -163,33 +179,6 @@ CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers)
return pcxt; 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 * Establish the dynamic shared memory segment for a parallel context and
* copy state and other bookkeeping information that will be needed by * copy state and other bookkeeping information that will be needed by
@ -249,15 +238,10 @@ InitializeParallelDSM(ParallelContext *pcxt)
pcxt->nworkers)); pcxt->nworkers));
shm_toc_estimate_keys(&pcxt->estimator, 1); shm_toc_estimate_keys(&pcxt->estimator, 1);
/* Estimate how much we'll need for extension entrypoint info. */ /* Estimate how much we'll need for the entrypoint info. */
if (pcxt->library_name != NULL) shm_toc_estimate_chunk(&pcxt->estimator, strlen(pcxt->library_name) +
{ strlen(pcxt->function_name) + 2);
Assert(pcxt->entrypoint == ParallelExtensionTrampoline); shm_toc_estimate_keys(&pcxt->estimator, 1);
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);
}
} }
/* /*
@ -297,7 +281,6 @@ InitializeParallelDSM(ParallelContext *pcxt)
fps->parallel_master_pgproc = MyProc; fps->parallel_master_pgproc = MyProc;
fps->parallel_master_pid = MyProcPid; fps->parallel_master_pid = MyProcPid;
fps->parallel_master_backend_id = MyBackendId; fps->parallel_master_backend_id = MyBackendId;
fps->entrypoint = pcxt->entrypoint;
SpinLockInit(&fps->mutex); SpinLockInit(&fps->mutex);
fps->last_xlog_end = 0; fps->last_xlog_end = 0;
shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps); shm_toc_insert(pcxt->toc, PARALLEL_KEY_FIXED, fps);
@ -312,6 +295,8 @@ InitializeParallelDSM(ParallelContext *pcxt)
char *asnapspace; char *asnapspace;
char *tstatespace; char *tstatespace;
char *error_queue_space; char *error_queue_space;
char *entrypointstate;
Size lnamelen;
/* Serialize shared libraries we have loaded. */ /* Serialize shared libraries we have loaded. */
libraryspace = shm_toc_allocate(pcxt->toc, library_len); 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); shm_toc_insert(pcxt->toc, PARALLEL_KEY_ERROR_QUEUE, error_queue_space);
/* Serialize extension entrypoint information. */ /*
if (pcxt->library_name != NULL) * Serialize entrypoint information. It's unsafe to pass function
{ * pointers across processes, as the function pointer may be different
Size lnamelen = strlen(pcxt->library_name); * in each process in EXEC_BACKEND builds, so we always pass library
char *extensionstate; * and function name. (We use library name "postgres" for functions
* in the core backend.)
extensionstate = shm_toc_allocate(pcxt->toc, lnamelen */
+ strlen(pcxt->function_name) + 2); lnamelen = strlen(pcxt->library_name);
strcpy(extensionstate, pcxt->library_name); entrypointstate = shm_toc_allocate(pcxt->toc, lnamelen +
strcpy(extensionstate + lnamelen + 1, pcxt->function_name); strlen(pcxt->function_name) + 2);
shm_toc_insert(pcxt->toc, PARALLEL_KEY_EXTENSION_TRAMPOLINE, strcpy(entrypointstate, pcxt->library_name);
extensionstate); strcpy(entrypointstate + lnamelen + 1, pcxt->function_name);
} shm_toc_insert(pcxt->toc, PARALLEL_KEY_ENTRYPOINT, entrypointstate);
} }
/* Restore previous memory context. */ /* Restore previous memory context. */
@ -671,6 +656,8 @@ DestroyParallelContext(ParallelContext *pcxt)
} }
/* Free memory. */ /* Free memory. */
pfree(pcxt->library_name);
pfree(pcxt->function_name);
pfree(pcxt); pfree(pcxt);
} }
@ -941,6 +928,10 @@ ParallelWorkerMain(Datum main_arg)
shm_mq *mq; shm_mq *mq;
shm_mq_handle *mqh; shm_mq_handle *mqh;
char *libraryspace; char *libraryspace;
char *entrypointstate;
char *library_name;
char *function_name;
parallel_worker_main_type entrypt;
char *gucspace; char *gucspace;
char *combocidspace; char *combocidspace;
char *tsnapspace; char *tsnapspace;
@ -1040,6 +1031,18 @@ ParallelWorkerMain(Datum main_arg)
Assert(libraryspace != NULL); Assert(libraryspace != NULL);
RestoreLibraryState(libraryspace); 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. */ /* Restore database connection. */
BackgroundWorkerInitializeConnectionByOid(fps->database_id, BackgroundWorkerInitializeConnectionByOid(fps->database_id,
fps->authenticated_user_id); fps->authenticated_user_id);
@ -1102,11 +1105,8 @@ ParallelWorkerMain(Datum main_arg)
/* /*
* Time to do the real work: invoke the caller-supplied code. * 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. */ /* Must exit parallel mode to pop active snapshot. */
ExitParallelMode(); ExitParallelMode();
@ -1121,33 +1121,6 @@ ParallelWorkerMain(Datum main_arg)
pq_putmessage('X', NULL, 0); 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 * Update shared memory with the ending location of the last WAL record we
* wrote, if it's greater than the value already stored there. * 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; fps->last_xlog_end = last_xlog_end;
SpinLockRelease(&fps->mutex); 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);
}

View File

@ -112,8 +112,7 @@ static shm_mq_handle **ExecParallelSetupTupleQueues(ParallelContext *pcxt,
static bool ExecParallelRetrieveInstrumentation(PlanState *planstate, static bool ExecParallelRetrieveInstrumentation(PlanState *planstate,
SharedExecutorInstrumentation *instrumentation); SharedExecutorInstrumentation *instrumentation);
/* Helper functions that run in the parallel worker. */ /* Helper function that runs in the parallel worker. */
static void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
static DestReceiver *ExecParallelGetReceiver(dsm_segment *seg, shm_toc *toc); 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); pstmt_data = ExecSerializePlan(planstate->plan, estate);
/* Create a parallel context. */ /* Create a parallel context. */
pcxt = CreateParallelContext(ParallelQueryMain, nworkers); pcxt = CreateParallelContext("postgres", "ParallelQueryMain", nworkers);
pei->pcxt = pcxt; 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 * to do this are also stored in the dsm_segment and can be accessed through
* the shm_toc. * the shm_toc.
*/ */
static void void
ParallelQueryMain(dsm_segment *seg, shm_toc *toc) ParallelQueryMain(dsm_segment *seg, shm_toc *toc)
{ {
BufferUsage *buffer_usage; BufferUsage *buffer_usage;

View File

@ -111,25 +111,30 @@ struct BackgroundWorkerHandle
static BackgroundWorkerArray *BackgroundWorkerData; static BackgroundWorkerArray *BackgroundWorkerData;
/* /*
* List of internal background workers. These are used for mapping the * List of internal background worker entry points. We need this for
* function name to actual function when building with EXEC_BACKEND and also * reasons explained in LookupBackgroundWorkerFunction(), below.
* to allow these to be loaded outside of shared_preload_libraries.
*/ */
typedef struct InternalBGWorkerMain static const struct
{ {
char *bgw_function_name; const char *fn_name;
bgworker_main_type bgw_main; bgworker_main_type fn_addr;
} InternalBGWorkerMain; } InternalBGWorkers[] =
static const InternalBGWorkerMain InternalBGWorkers[] = { {
{"ParallelWorkerMain", ParallelWorkerMain}, {
{"ApplyLauncherMain", ApplyLauncherMain}, "ParallelWorkerMain", ParallelWorkerMain
{"ApplyWorkerMain", ApplyWorkerMain}, },
/* Dummy entry marking end of the array. */ {
{NULL, NULL} "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. * Calculate shared memory needed.
@ -812,18 +817,10 @@ StartBackgroundWorker(void)
} }
/* /*
* For internal workers set the entry point to known function address. * Look up the entry point function, loading its library if necessary.
* 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).
*/ */
entrypt = GetInternalBgWorkerMain(worker); entrypt = LookupBackgroundWorkerFunction(worker->bgw_library_name,
worker->bgw_function_name);
if (entrypt == NULL)
entrypt = (bgworker_main_type)
load_external_function(worker->bgw_library_name,
worker->bgw_function_name,
true, NULL);
/* /*
* Note that in normal processes, we would call InitPostgres here. For a * 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 * This can only be called directly from postmaster or in the _PG_init
* that's loaded by shared_preload_libraries; otherwise it has no effect. * function of a module library that's loaded by shared_preload_libraries;
* otherwise it will have no effect.
*/ */
void void
RegisterBackgroundWorker(BackgroundWorker *worker) RegisterBackgroundWorker(BackgroundWorker *worker)
@ -858,7 +856,7 @@ RegisterBackgroundWorker(BackgroundWorker *worker)
(errmsg("registering background worker \"%s\"", worker->bgw_name))); (errmsg("registering background worker \"%s\"", worker->bgw_name)));
if (!process_shared_preload_libraries_in_progress && if (!process_shared_preload_libraries_in_progress &&
GetInternalBgWorkerMain(worker) == NULL) strcmp(worker->bgw_library_name, "postgres") != 0)
{ {
if (!IsUnderPostmaster) if (!IsUnderPostmaster)
ereport(LOG, ereport(LOG,
@ -1193,26 +1191,45 @@ TerminateBackgroundWorker(BackgroundWorkerHandle *handle)
} }
/* /*
* Search the known internal worker array and return its main function * Look up (and possibly load) a bgworker entry point function.
* pointer if found.
* *
* Returns NULL if not known internal worker. * 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().
*
* 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 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
/* Internal workers always have to use postgres as library name. */ * InternalBGWorkers array.
if (strncmp(worker->bgw_library_name, "postgres", BGW_MAXLEN) != 0) */
return NULL; if (strcmp(libraryname, "postgres") == 0)
for (i = 0; InternalBGWorkers[i].bgw_function_name; i++)
{ {
if (strncmp(InternalBGWorkers[i].bgw_function_name, int i;
worker->bgw_function_name, BGW_MAXLEN) == 0)
return InternalBGWorkers[i].bgw_main; for (i = 0; i < lengthof(InternalBGWorkers); i++)
{
if (strcmp(InternalBGWorkers[i].fn_name, funcname) == 0)
return InternalBGWorkers[i].fn_addr;
}
/* 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);
} }

View File

@ -91,7 +91,7 @@ static const Pg_magic_struct magic_data = PG_MODULE_MAGIC_DATA;
* at less cost than repeating load_external_function. * at less cost than repeating load_external_function.
*/ */
PGFunction PGFunction
load_external_function(char *filename, char *funcname, load_external_function(const char *filename, const char *funcname,
bool signalNotFound, void **filehandle) bool signalNotFound, void **filehandle)
{ {
char *fullname; char *fullname;
@ -108,8 +108,12 @@ load_external_function(char *filename, char *funcname,
if (filehandle) if (filehandle)
*filehandle = lib_handle; *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) if (retval == NULL && signalNotFound)
ereport(ERROR, ereport(ERROR,
@ -155,9 +159,10 @@ load_file(const char *filename, bool restricted)
* Return (PGFunction) NULL if not found. * Return (PGFunction) NULL if not found.
*/ */
PGFunction 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);
} }

View File

@ -381,7 +381,7 @@ fmgr_info_other_lang(Oid functionId, FmgrInfo *finfo, HeapTuple procedureTuple)
* pg_proc. * pg_proc.
*/ */
const Pg_finfo_record * const Pg_finfo_record *
fetch_finfo_record(void *filehandle, char *funcname) fetch_finfo_record(void *filehandle, const char *funcname)
{ {
char *infofuncname; char *infofuncname;
PGFInfoFunction infofunc; PGFInfoFunction infofunc;

View File

@ -35,7 +35,6 @@ typedef struct ParallelContext
SubTransactionId subid; SubTransactionId subid;
int nworkers; int nworkers;
int nworkers_launched; int nworkers_launched;
parallel_worker_main_type entrypoint;
char *library_name; char *library_name;
char *function_name; char *function_name;
ErrorContextCallback *error_context_stack; ErrorContextCallback *error_context_stack;
@ -52,8 +51,7 @@ extern bool InitializingParallelWorker;
#define IsParallelWorker() (ParallelWorkerNumber >= 0) #define IsParallelWorker() (ParallelWorkerNumber >= 0)
extern ParallelContext *CreateParallelContext(parallel_worker_main_type entrypoint, int nworkers); extern ParallelContext *CreateParallelContext(const char *library_name, const char *function_name, int nworkers);
extern ParallelContext *CreateParallelContextForExternalFunction(char *library_name, char *function_name, int nworkers);
extern void InitializeParallelDSM(ParallelContext *pcxt); extern void InitializeParallelDSM(ParallelContext *pcxt);
extern void ReinitializeParallelDSM(ParallelContext *pcxt); extern void ReinitializeParallelDSM(ParallelContext *pcxt);
extern void LaunchParallelWorkers(ParallelContext *pcxt); extern void LaunchParallelWorkers(ParallelContext *pcxt);

View File

@ -38,4 +38,6 @@ extern void ExecParallelFinish(ParallelExecutorInfo *pei);
extern void ExecParallelCleanup(ParallelExecutorInfo *pei); extern void ExecParallelCleanup(ParallelExecutorInfo *pei);
extern void ExecParallelReinitialize(ParallelExecutorInfo *pei); extern void ExecParallelReinitialize(ParallelExecutorInfo *pei);
extern void ParallelQueryMain(dsm_segment *seg, shm_toc *toc);
#endif /* EXECPARALLEL_H */ #endif /* EXECPARALLEL_H */

View File

@ -657,7 +657,7 @@ extern bytea *OidSendFunctionCall(Oid functionId, Datum val);
/* /*
* Routines in fmgr.c * 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 void clear_external_function_hash(void *filehandle);
extern Oid fmgr_internal_function(const char *proname); extern Oid fmgr_internal_function(const char *proname);
extern Oid get_fn_expr_rettype(FmgrInfo *flinfo); 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 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); 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 load_file(const char *filename, bool restricted);
extern void **find_rendezvous_variable(const char *varName); extern void **find_rendezvous_variable(const char *varName);
extern Size EstimateLibraryStateSpace(void); extern Size EstimateLibraryStateSpace(void);