mirror of
https://github.com/postgres/postgres.git
synced 2025-11-04 20:11:56 +03:00
Replace BackendIds with 0-based ProcNumbers
Now that BackendId was just another index into the proc array, it was redundant with the 0-based proc numbers used in other places. Replace all usage of backend IDs with proc numbers. The only place where the term "backend id" remains is in a few pgstat functions that expose backend IDs at the SQL level. Those IDs are now in fact 0-based ProcNumbers too, but the documentation still calls them "backend ids". That term still seems appropriate to describe what the numbers are, so I let it be. One user-visible effect is that pg_temp_0 is now a valid temp schema name, for backend with ProcNumber 0. Reviewed-by: Andres Freund Discussion: https://www.postgresql.org/message-id/8171f1aa-496f-46a6-afc3-c46fe7a9b407@iki.fi
This commit is contained in:
@@ -122,7 +122,7 @@ InitBufferPool(void)
|
||||
ClearBufferTag(&buf->tag);
|
||||
|
||||
pg_atomic_init_u32(&buf->state, 0);
|
||||
buf->wait_backend_pgprocno = INVALID_PGPROCNO;
|
||||
buf->wait_backend_pgprocno = INVALID_PROC_NUMBER;
|
||||
|
||||
buf->buf_id = i;
|
||||
|
||||
|
||||
@@ -816,7 +816,7 @@ ReadBufferExtended(Relation reln, ForkNumber forkNum, BlockNumber blockNum,
|
||||
* permanent = false for a RELPERSISTENCE_UNLOGGED relation. This function
|
||||
* cannot be used for temporary relations (and making that work might be
|
||||
* difficult, unless we only want to read temporary relations for our own
|
||||
* BackendId).
|
||||
* ProcNumber).
|
||||
*/
|
||||
Buffer
|
||||
ReadBufferWithoutRelcache(RelFileLocator rlocator, ForkNumber forkNum,
|
||||
@@ -825,7 +825,7 @@ ReadBufferWithoutRelcache(RelFileLocator rlocator, ForkNumber forkNum,
|
||||
{
|
||||
bool hit;
|
||||
|
||||
SMgrRelation smgr = smgropen(rlocator, InvalidBackendId);
|
||||
SMgrRelation smgr = smgropen(rlocator, INVALID_PROC_NUMBER);
|
||||
|
||||
return ReadBuffer_common(smgr, permanent ? RELPERSISTENCE_PERMANENT :
|
||||
RELPERSISTENCE_UNLOGGED, forkNum, blockNum,
|
||||
@@ -3322,7 +3322,7 @@ DebugPrintBufferRefcount(Buffer buffer)
|
||||
int32 loccount;
|
||||
char *path;
|
||||
char *result;
|
||||
BackendId backend;
|
||||
ProcNumber backend;
|
||||
uint32 buf_state;
|
||||
|
||||
Assert(BufferIsValid(buffer));
|
||||
@@ -3330,13 +3330,13 @@ DebugPrintBufferRefcount(Buffer buffer)
|
||||
{
|
||||
buf = GetLocalBufferDescriptor(-buffer - 1);
|
||||
loccount = LocalRefCount[-buffer - 1];
|
||||
backend = MyBackendId;
|
||||
backend = MyProcNumber;
|
||||
}
|
||||
else
|
||||
{
|
||||
buf = GetBufferDescriptor(buffer - 1);
|
||||
loccount = GetPrivateRefCount(buffer);
|
||||
backend = InvalidBackendId;
|
||||
backend = INVALID_PROC_NUMBER;
|
||||
}
|
||||
|
||||
/* theoretically we should lock the bufhdr here */
|
||||
@@ -3461,7 +3461,7 @@ FlushBuffer(BufferDesc *buf, SMgrRelation reln, IOObject io_object,
|
||||
|
||||
/* Find smgr relation for buffer */
|
||||
if (reln == NULL)
|
||||
reln = smgropen(BufTagGetRelFileLocator(&buf->tag), InvalidBackendId);
|
||||
reln = smgropen(BufTagGetRelFileLocator(&buf->tag), INVALID_PROC_NUMBER);
|
||||
|
||||
TRACE_POSTGRESQL_BUFFER_FLUSH_START(BufTagGetForkNum(&buf->tag),
|
||||
buf->tag.blockNum,
|
||||
@@ -3696,7 +3696,7 @@ DropRelationBuffers(SMgrRelation smgr_reln, ForkNumber *forkNum,
|
||||
/* If it's a local relation, it's localbuf.c's problem. */
|
||||
if (RelFileLocatorBackendIsTemp(rlocator))
|
||||
{
|
||||
if (rlocator.backend == MyBackendId)
|
||||
if (rlocator.backend == MyProcNumber)
|
||||
{
|
||||
for (j = 0; j < nforks; j++)
|
||||
DropRelationLocalBuffers(rlocator.locator, forkNum[j],
|
||||
@@ -3826,7 +3826,7 @@ DropRelationsAllBuffers(SMgrRelation *smgr_reln, int nlocators)
|
||||
{
|
||||
if (RelFileLocatorBackendIsTemp(smgr_reln[i]->smgr_rlocator))
|
||||
{
|
||||
if (smgr_reln[i]->smgr_rlocator.backend == MyBackendId)
|
||||
if (smgr_reln[i]->smgr_rlocator.backend == MyProcNumber)
|
||||
DropRelationAllLocalBuffers(smgr_reln[i]->smgr_rlocator.locator);
|
||||
}
|
||||
else
|
||||
@@ -4090,7 +4090,7 @@ PrintBufferDescs(void)
|
||||
"blockNum=%u, flags=0x%x, refcount=%u %d)",
|
||||
i, buf->freeNext,
|
||||
relpathbackend(BufTagGetRelFileLocator(&buf->tag),
|
||||
InvalidBackendId, BufTagGetForkNum(&buf->tag)),
|
||||
INVALID_PROC_NUMBER, BufTagGetForkNum(&buf->tag)),
|
||||
buf->tag.blockNum, buf->flags,
|
||||
buf->refcount, GetPrivateRefCount(b));
|
||||
}
|
||||
@@ -4364,7 +4364,7 @@ RelationCopyStorageUsingBuffer(RelFileLocator srclocator,
|
||||
use_wal = XLogIsNeeded() && (permanent || forkNum == INIT_FORKNUM);
|
||||
|
||||
/* Get number of blocks in the source relation. */
|
||||
nblocks = smgrnblocks(smgropen(srclocator, InvalidBackendId),
|
||||
nblocks = smgrnblocks(smgropen(srclocator, INVALID_PROC_NUMBER),
|
||||
forkNum);
|
||||
|
||||
/* Nothing to copy; just return. */
|
||||
@@ -4376,7 +4376,7 @@ RelationCopyStorageUsingBuffer(RelFileLocator srclocator,
|
||||
* relation before starting to copy block by block.
|
||||
*/
|
||||
memset(buf.data, 0, BLCKSZ);
|
||||
smgrextend(smgropen(dstlocator, InvalidBackendId), forkNum, nblocks - 1,
|
||||
smgrextend(smgropen(dstlocator, INVALID_PROC_NUMBER), forkNum, nblocks - 1,
|
||||
buf.data, true);
|
||||
|
||||
/* This is a bulk operation, so use buffer access strategies. */
|
||||
@@ -4443,8 +4443,8 @@ CreateAndCopyRelationData(RelFileLocator src_rlocator,
|
||||
relpersistence = permanent ?
|
||||
RELPERSISTENCE_PERMANENT : RELPERSISTENCE_UNLOGGED;
|
||||
|
||||
src_rel = smgropen(src_rlocator, InvalidBackendId);
|
||||
dst_rel = smgropen(dst_rlocator, InvalidBackendId);
|
||||
src_rel = smgropen(src_rlocator, INVALID_PROC_NUMBER);
|
||||
dst_rel = smgropen(dst_rlocator, INVALID_PROC_NUMBER);
|
||||
|
||||
/*
|
||||
* Create and copy all forks of the relation. During create database we
|
||||
@@ -5348,7 +5348,7 @@ local_buffer_write_error_callback(void *arg)
|
||||
if (bufHdr != NULL)
|
||||
{
|
||||
char *path = relpathbackend(BufTagGetRelFileLocator(&bufHdr->tag),
|
||||
MyBackendId,
|
||||
MyProcNumber,
|
||||
BufTagGetForkNum(&bufHdr->tag));
|
||||
|
||||
errcontext("writing block %u of relation %s",
|
||||
@@ -5653,7 +5653,7 @@ IssuePendingWritebacks(WritebackContext *wb_context, IOContext io_context)
|
||||
i += ahead;
|
||||
|
||||
/* and finally tell the kernel to write the data to storage */
|
||||
reln = smgropen(currlocator, InvalidBackendId);
|
||||
reln = smgropen(currlocator, INVALID_PROC_NUMBER);
|
||||
smgrwriteback(reln, BufTagGetForkNum(&tag), tag.blockNum, nblocks);
|
||||
}
|
||||
|
||||
|
||||
@@ -242,7 +242,7 @@ GetLocalVictimBuffer(void)
|
||||
Page localpage = (char *) LocalBufHdrGetBlock(bufHdr);
|
||||
|
||||
/* Find smgr relation for buffer */
|
||||
oreln = smgropen(BufTagGetRelFileLocator(&bufHdr->tag), MyBackendId);
|
||||
oreln = smgropen(BufTagGetRelFileLocator(&bufHdr->tag), MyProcNumber);
|
||||
|
||||
PageSetChecksumInplace(localpage, bufHdr->tag.blockNum);
|
||||
|
||||
@@ -509,7 +509,7 @@ DropRelationLocalBuffers(RelFileLocator rlocator, ForkNumber forkNum,
|
||||
elog(ERROR, "block %u of %s is still referenced (local %u)",
|
||||
bufHdr->tag.blockNum,
|
||||
relpathbackend(BufTagGetRelFileLocator(&bufHdr->tag),
|
||||
MyBackendId,
|
||||
MyProcNumber,
|
||||
BufTagGetForkNum(&bufHdr->tag)),
|
||||
LocalRefCount[i]);
|
||||
|
||||
@@ -554,7 +554,7 @@ DropRelationAllLocalBuffers(RelFileLocator rlocator)
|
||||
elog(ERROR, "block %u of %s is still referenced (local %u)",
|
||||
bufHdr->tag.blockNum,
|
||||
relpathbackend(BufTagGetRelFileLocator(&bufHdr->tag),
|
||||
MyBackendId,
|
||||
MyProcNumber,
|
||||
BufTagGetForkNum(&bufHdr->tag)),
|
||||
LocalRefCount[i]);
|
||||
/* Remove entry from hashtable */
|
||||
|
||||
@@ -818,9 +818,9 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
|
||||
* If the list was not empty, the leader will clear our XID. It is
|
||||
* impossible to have followers without a leader because the first process
|
||||
* that has added itself to the list will always have nextidx as
|
||||
* INVALID_PGPROCNO.
|
||||
* INVALID_PROC_NUMBER.
|
||||
*/
|
||||
if (nextidx != INVALID_PGPROCNO)
|
||||
if (nextidx != INVALID_PROC_NUMBER)
|
||||
{
|
||||
int extraWaits = 0;
|
||||
|
||||
@@ -836,7 +836,7 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
|
||||
}
|
||||
pgstat_report_wait_end();
|
||||
|
||||
Assert(pg_atomic_read_u32(&proc->procArrayGroupNext) == INVALID_PGPROCNO);
|
||||
Assert(pg_atomic_read_u32(&proc->procArrayGroupNext) == INVALID_PROC_NUMBER);
|
||||
|
||||
/* Fix semaphore count for any absorbed wakeups */
|
||||
while (extraWaits-- > 0)
|
||||
@@ -853,13 +853,13 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
|
||||
* to pop elements one at a time could lead to an ABA problem.
|
||||
*/
|
||||
nextidx = pg_atomic_exchange_u32(&procglobal->procArrayGroupFirst,
|
||||
INVALID_PGPROCNO);
|
||||
INVALID_PROC_NUMBER);
|
||||
|
||||
/* Remember head of list so we can perform wakeups after dropping lock. */
|
||||
wakeidx = nextidx;
|
||||
|
||||
/* Walk the list and clear all XIDs. */
|
||||
while (nextidx != INVALID_PGPROCNO)
|
||||
while (nextidx != INVALID_PROC_NUMBER)
|
||||
{
|
||||
PGPROC *nextproc = &allProcs[nextidx];
|
||||
|
||||
@@ -879,12 +879,12 @@ ProcArrayGroupClearXid(PGPROC *proc, TransactionId latestXid)
|
||||
* up are probably much slower than the simple memory writes we did while
|
||||
* holding the lock.
|
||||
*/
|
||||
while (wakeidx != INVALID_PGPROCNO)
|
||||
while (wakeidx != INVALID_PROC_NUMBER)
|
||||
{
|
||||
PGPROC *nextproc = &allProcs[wakeidx];
|
||||
|
||||
wakeidx = pg_atomic_read_u32(&nextproc->procArrayGroupNext);
|
||||
pg_atomic_write_u32(&nextproc->procArrayGroupNext, INVALID_PGPROCNO);
|
||||
pg_atomic_write_u32(&nextproc->procArrayGroupNext, INVALID_PROC_NUMBER);
|
||||
|
||||
/* ensure all previous writes are visible before follower continues. */
|
||||
pg_write_barrier();
|
||||
@@ -2538,7 +2538,7 @@ ProcArrayInstallImportedXmin(TransactionId xmin,
|
||||
|
||||
/*
|
||||
* Find the PGPROC entry of the source transaction. (This could use
|
||||
* GetPGProcByBackendId(), unless it's a prepared xact. But this isn't
|
||||
* GetPGProcByNumber(), unless it's a prepared xact. But this isn't
|
||||
* performance critical.)
|
||||
*/
|
||||
for (index = 0; index < arrayP->numProcs; index++)
|
||||
@@ -2553,7 +2553,7 @@ ProcArrayInstallImportedXmin(TransactionId xmin,
|
||||
continue;
|
||||
|
||||
/* We are only interested in the specific virtual transaction. */
|
||||
if (proc->vxid.backendId != sourcevxid->backendId)
|
||||
if (proc->vxid.procNumber != sourcevxid->procNumber)
|
||||
continue;
|
||||
if (proc->vxid.lxid != sourcevxid->localTransactionId)
|
||||
continue;
|
||||
@@ -3105,20 +3105,20 @@ HaveVirtualXIDsDelayingChkpt(VirtualTransactionId *vxids, int nvxids, int type)
|
||||
}
|
||||
|
||||
/*
|
||||
* BackendIdGetProc -- get a backend's PGPROC given its backend ID
|
||||
* ProcNumberGetProc -- get a backend's PGPROC given its proc number
|
||||
*
|
||||
* The result may be out of date arbitrarily quickly, so the caller
|
||||
* must be careful about how this information is used. NULL is
|
||||
* returned if the backend is not active.
|
||||
*/
|
||||
PGPROC *
|
||||
BackendIdGetProc(int backendID)
|
||||
ProcNumberGetProc(ProcNumber procNumber)
|
||||
{
|
||||
PGPROC *result;
|
||||
|
||||
if (backendID < 1 || backendID > ProcGlobal->allProcCount)
|
||||
if (procNumber < 0 || procNumber >= ProcGlobal->allProcCount)
|
||||
return NULL;
|
||||
result = GetPGProcByBackendId(backendID);
|
||||
result = GetPGProcByNumber(procNumber);
|
||||
|
||||
if (result->pid == 0)
|
||||
return NULL;
|
||||
@@ -3127,15 +3127,15 @@ BackendIdGetProc(int backendID)
|
||||
}
|
||||
|
||||
/*
|
||||
* BackendIdGetTransactionIds -- get a backend's transaction status
|
||||
* ProcNumberGetTransactionIds -- get a backend's transaction status
|
||||
*
|
||||
* Get the xid, xmin, nsubxid and overflow status of the backend. The
|
||||
* result may be out of date arbitrarily quickly, so the caller must be
|
||||
* careful about how this information is used.
|
||||
*/
|
||||
void
|
||||
BackendIdGetTransactionIds(int backendID, TransactionId *xid,
|
||||
TransactionId *xmin, int *nsubxid, bool *overflowed)
|
||||
ProcNumberGetTransactionIds(ProcNumber procNumber, TransactionId *xid,
|
||||
TransactionId *xmin, int *nsubxid, bool *overflowed)
|
||||
{
|
||||
PGPROC *proc;
|
||||
|
||||
@@ -3144,9 +3144,9 @@ BackendIdGetTransactionIds(int backendID, TransactionId *xid,
|
||||
*nsubxid = 0;
|
||||
*overflowed = false;
|
||||
|
||||
if (backendID < 1 || backendID > ProcGlobal->allProcCount)
|
||||
if (procNumber < 0 || procNumber >= ProcGlobal->allProcCount)
|
||||
return;
|
||||
proc = GetPGProcByBackendId(backendID);
|
||||
proc = GetPGProcByNumber(procNumber);
|
||||
|
||||
/* Need to lock out additions/removals of backends */
|
||||
LWLockAcquire(ProcArrayLock, LW_SHARED);
|
||||
@@ -3453,7 +3453,7 @@ GetConflictingVirtualXIDs(TransactionId limitXmin, Oid dbOid)
|
||||
LWLockRelease(ProcArrayLock);
|
||||
|
||||
/* add the terminator */
|
||||
vxids[count].backendId = InvalidBackendId;
|
||||
vxids[count].procNumber = INVALID_PROC_NUMBER;
|
||||
vxids[count].localTransactionId = InvalidLocalTransactionId;
|
||||
|
||||
return vxids;
|
||||
@@ -3488,7 +3488,7 @@ SignalVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode,
|
||||
|
||||
GET_VXID_FROM_PGPROC(procvxid, *proc);
|
||||
|
||||
if (procvxid.backendId == vxid.backendId &&
|
||||
if (procvxid.procNumber == vxid.procNumber &&
|
||||
procvxid.localTransactionId == vxid.localTransactionId)
|
||||
{
|
||||
proc->recoveryConflictPending = conflictPending;
|
||||
@@ -3499,7 +3499,7 @@ SignalVirtualTransaction(VirtualTransactionId vxid, ProcSignalReason sigmode,
|
||||
* Kill the pid if it's still here. If not, that's what we
|
||||
* wanted so ignore any errors.
|
||||
*/
|
||||
(void) SendProcSignal(pid, sigmode, vxid.backendId);
|
||||
(void) SendProcSignal(pid, sigmode, vxid.procNumber);
|
||||
}
|
||||
break;
|
||||
}
|
||||
@@ -3662,7 +3662,7 @@ CancelDBBackends(Oid databaseid, ProcSignalReason sigmode, bool conflictPending)
|
||||
* Kill the pid if it's still here. If not, that's what we
|
||||
* wanted so ignore any errors.
|
||||
*/
|
||||
(void) SendProcSignal(pid, sigmode, procvxid.backendId);
|
||||
(void) SendProcSignal(pid, sigmode, procvxid.procNumber);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -43,10 +43,10 @@
|
||||
* observe it only once.)
|
||||
*
|
||||
* Each process that wants to receive signals registers its process ID
|
||||
* in the ProcSignalSlots array. The array is indexed by backend ID to make
|
||||
* in the ProcSignalSlots array. The array is indexed by ProcNumber to make
|
||||
* slot allocation simple, and to avoid having to search the array when you
|
||||
* know the backend ID of the process you're signaling. (We do support
|
||||
* signaling without backend ID, but it's a bit less efficient.)
|
||||
* know the ProcNumber of the process you're signaling. (We do support
|
||||
* signaling without ProcNumber, but it's a bit less efficient.)
|
||||
*
|
||||
* The flags are actually declared as "volatile sig_atomic_t" for maximum
|
||||
* portability. This should ensure that loads and stores of the flag
|
||||
@@ -83,7 +83,7 @@ typedef struct
|
||||
} ProcSignalHeader;
|
||||
|
||||
/*
|
||||
* We reserve a slot for each possible BackendId, plus one for each
|
||||
* We reserve a slot for each possible ProcNumber, plus one for each
|
||||
* possible auxiliary process type. (This scheme assumes there is not
|
||||
* more than one of any auxiliary process type at a time.)
|
||||
*/
|
||||
@@ -161,16 +161,16 @@ ProcSignalInit(void)
|
||||
ProcSignalSlot *slot;
|
||||
uint64 barrier_generation;
|
||||
|
||||
if (MyBackendId <= 0)
|
||||
elog(ERROR, "MyBackendId not set");
|
||||
if (MyBackendId > NumProcSignalSlots)
|
||||
elog(ERROR, "unexpected MyBackendId %d in ProcSignalInit (max %d)", MyBackendId, NumProcSignalSlots);
|
||||
slot = &ProcSignal->psh_slot[MyBackendId - 1];
|
||||
if (MyProcNumber < 0)
|
||||
elog(ERROR, "MyProcNumber not set");
|
||||
if (MyProcNumber >= NumProcSignalSlots)
|
||||
elog(ERROR, "unexpected MyProcNumber %d in ProcSignalInit (max %d)", MyProcNumber, NumProcSignalSlots);
|
||||
slot = &ProcSignal->psh_slot[MyProcNumber];
|
||||
|
||||
/* sanity check */
|
||||
if (slot->pss_pid != 0)
|
||||
elog(LOG, "process %d taking over ProcSignal slot %d, but it's not empty",
|
||||
MyProcPid, MyBackendId - 1);
|
||||
MyProcPid, MyProcNumber);
|
||||
|
||||
/* Clear out any leftover signal reasons */
|
||||
MemSet(slot->pss_signalFlags, 0, NUM_PROCSIGNALS * sizeof(sig_atomic_t));
|
||||
@@ -218,6 +218,7 @@ CleanupProcSignalState(int status, Datum arg)
|
||||
* won't try to access it after it's no longer ours (and perhaps even
|
||||
* after we've unmapped the shared memory segment).
|
||||
*/
|
||||
Assert(MyProcSignalSlot != NULL);
|
||||
MyProcSignalSlot = NULL;
|
||||
|
||||
/* sanity check */
|
||||
@@ -246,7 +247,7 @@ CleanupProcSignalState(int status, Datum arg)
|
||||
* SendProcSignal
|
||||
* Send a signal to a Postgres process
|
||||
*
|
||||
* Providing backendId is optional, but it will speed up the operation.
|
||||
* Providing procNumber is optional, but it will speed up the operation.
|
||||
*
|
||||
* On success (a signal was sent), zero is returned.
|
||||
* On error, -1 is returned, and errno is set (typically to ESRCH or EPERM).
|
||||
@@ -254,13 +255,13 @@ CleanupProcSignalState(int status, Datum arg)
|
||||
* Not to be confused with ProcSendSignal
|
||||
*/
|
||||
int
|
||||
SendProcSignal(pid_t pid, ProcSignalReason reason, BackendId backendId)
|
||||
SendProcSignal(pid_t pid, ProcSignalReason reason, ProcNumber procNumber)
|
||||
{
|
||||
volatile ProcSignalSlot *slot;
|
||||
|
||||
if (backendId != InvalidBackendId)
|
||||
if (procNumber != INVALID_PROC_NUMBER)
|
||||
{
|
||||
slot = &ProcSignal->psh_slot[backendId - 1];
|
||||
slot = &ProcSignal->psh_slot[procNumber];
|
||||
|
||||
/*
|
||||
* Note: Since there's no locking, it's possible that the target
|
||||
@@ -281,10 +282,11 @@ SendProcSignal(pid_t pid, ProcSignalReason reason, BackendId backendId)
|
||||
else
|
||||
{
|
||||
/*
|
||||
* BackendId not provided, so search the array using pid. We search
|
||||
* Pronumber not provided, so search the array using pid. We search
|
||||
* the array back to front so as to reduce search overhead. Passing
|
||||
* InvalidBackendId means that the target is most likely an auxiliary
|
||||
* process, which will have a slot near the end of the array.
|
||||
* INVALID_PROC_NUMBER means that the target is most likely an
|
||||
* auxiliary process, which will have a slot near the end of the
|
||||
* array.
|
||||
*/
|
||||
int i;
|
||||
|
||||
|
||||
@@ -19,9 +19,9 @@
|
||||
|
||||
#include "access/transam.h"
|
||||
#include "miscadmin.h"
|
||||
#include "storage/backendid.h"
|
||||
#include "storage/ipc.h"
|
||||
#include "storage/proc.h"
|
||||
#include "storage/procnumber.h"
|
||||
#include "storage/procsignal.h"
|
||||
#include "storage/shmem.h"
|
||||
#include "storage/sinvaladt.h"
|
||||
@@ -155,8 +155,8 @@ typedef struct ProcState
|
||||
|
||||
/*
|
||||
* Next LocalTransactionId to use for each idle backend slot. We keep
|
||||
* this here because it is indexed by BackendId and it is convenient to
|
||||
* copy the value to and from local memory when MyBackendId is set. It's
|
||||
* this here because it is indexed by ProcNumber and it is convenient to
|
||||
* copy the value to and from local memory when MyProcNumber is set. It's
|
||||
* meaningless in an active ProcState entry.
|
||||
*/
|
||||
LocalTransactionId nextLXID;
|
||||
@@ -197,7 +197,7 @@ typedef struct SISeg
|
||||
} SISeg;
|
||||
|
||||
/*
|
||||
* We reserve a slot for each possible BackendId, plus one for each
|
||||
* We reserve a slot for each possible ProcNumber, plus one for each
|
||||
* possible auxiliary process type. (This scheme assumes there is not
|
||||
* more than one of any auxiliary process type at a time.)
|
||||
*/
|
||||
@@ -274,15 +274,13 @@ SharedInvalBackendInit(bool sendOnly)
|
||||
ProcState *stateP;
|
||||
pid_t oldPid;
|
||||
SISeg *segP = shmInvalBuffer;
|
||||
int pgprocno;
|
||||
|
||||
if (MyBackendId <= 0)
|
||||
elog(ERROR, "MyBackendId not set");
|
||||
if (MyBackendId > NumProcStateSlots)
|
||||
elog(PANIC, "unexpected MyBackendId %d in SharedInvalBackendInit (max %d)",
|
||||
MyBackendId, NumProcStateSlots);
|
||||
pgprocno = MyBackendId - 1;
|
||||
stateP = &segP->procState[pgprocno];
|
||||
if (MyProcNumber < 0)
|
||||
elog(ERROR, "MyProcNumber not set");
|
||||
if (MyProcNumber >= NumProcStateSlots)
|
||||
elog(PANIC, "unexpected MyProcNumber %d in SharedInvalBackendInit (max %d)",
|
||||
MyProcNumber, NumProcStateSlots);
|
||||
stateP = &segP->procState[MyProcNumber];
|
||||
|
||||
/*
|
||||
* This can run in parallel with read operations, but not with write
|
||||
@@ -296,10 +294,10 @@ SharedInvalBackendInit(bool sendOnly)
|
||||
{
|
||||
LWLockRelease(SInvalWriteLock);
|
||||
elog(ERROR, "sinval slot for backend %d is already in use by process %d",
|
||||
MyBackendId, (int) oldPid);
|
||||
MyProcNumber, (int) oldPid);
|
||||
}
|
||||
|
||||
shmInvalBuffer->pgprocnos[shmInvalBuffer->numProcs++] = pgprocno;
|
||||
shmInvalBuffer->pgprocnos[shmInvalBuffer->numProcs++] = MyProcNumber;
|
||||
|
||||
/* Fetch next local transaction ID into local memory */
|
||||
nextLocalTransactionId = stateP->nextLXID;
|
||||
@@ -331,16 +329,15 @@ CleanupInvalidationState(int status, Datum arg)
|
||||
{
|
||||
SISeg *segP = (SISeg *) DatumGetPointer(arg);
|
||||
ProcState *stateP;
|
||||
int pgprocno = MyBackendId - 1;
|
||||
int i;
|
||||
|
||||
Assert(PointerIsValid(segP));
|
||||
|
||||
LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
|
||||
|
||||
stateP = &segP->procState[pgprocno];
|
||||
stateP = &segP->procState[MyProcNumber];
|
||||
|
||||
/* Update next local transaction ID for next holder of this backendID */
|
||||
/* Update next local transaction ID for next holder of this proc number */
|
||||
stateP->nextLXID = nextLocalTransactionId;
|
||||
|
||||
/* Mark myself inactive */
|
||||
@@ -351,7 +348,7 @@ CleanupInvalidationState(int status, Datum arg)
|
||||
|
||||
for (i = segP->numProcs - 1; i >= 0; i--)
|
||||
{
|
||||
if (segP->pgprocnos[i] == pgprocno)
|
||||
if (segP->pgprocnos[i] == MyProcNumber)
|
||||
{
|
||||
if (i != segP->numProcs - 1)
|
||||
segP->pgprocnos[i] = segP->pgprocnos[segP->numProcs - 1];
|
||||
@@ -481,7 +478,7 @@ SIGetDataEntries(SharedInvalidationMessage *data, int datasize)
|
||||
int n;
|
||||
|
||||
segP = shmInvalBuffer;
|
||||
stateP = &segP->procState[MyBackendId - 1];
|
||||
stateP = &segP->procState[MyProcNumber];
|
||||
|
||||
/*
|
||||
* Before starting to take locks, do a quick, unlocked test to see whether
|
||||
@@ -668,13 +665,13 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
|
||||
if (needSig)
|
||||
{
|
||||
pid_t his_pid = needSig->procPid;
|
||||
BackendId his_backendId = (needSig - &segP->procState[0]) + 1;
|
||||
ProcNumber his_procNumber = (needSig - &segP->procState[0]);
|
||||
|
||||
needSig->signaled = true;
|
||||
LWLockRelease(SInvalReadLock);
|
||||
LWLockRelease(SInvalWriteLock);
|
||||
elog(DEBUG4, "sending sinval catchup signal to PID %d", (int) his_pid);
|
||||
SendProcSignal(his_pid, PROCSIG_CATCHUP_INTERRUPT, his_backendId);
|
||||
SendProcSignal(his_pid, PROCSIG_CATCHUP_INTERRUPT, his_procNumber);
|
||||
if (callerHasWriteLock)
|
||||
LWLockAcquire(SInvalWriteLock, LW_EXCLUSIVE);
|
||||
}
|
||||
@@ -693,11 +690,11 @@ SICleanupQueue(bool callerHasWriteLock, int minFree)
|
||||
* We split VirtualTransactionIds into two parts so that it is possible
|
||||
* to allocate a new one without any contention for shared memory, except
|
||||
* for a bit of additional overhead during backend startup/shutdown.
|
||||
* The high-order part of a VirtualTransactionId is a BackendId, and the
|
||||
* The high-order part of a VirtualTransactionId is a ProcNumber, and the
|
||||
* low-order part is a LocalTransactionId, which we assign from a local
|
||||
* counter. To avoid the risk of a VirtualTransactionId being reused
|
||||
* within a short interval, successive procs occupying the same backend ID
|
||||
* slot should use a consecutive sequence of local IDs, which is implemented
|
||||
* within a short interval, successive procs occupying the same PGPROC slot
|
||||
* should use a consecutive sequence of local IDs, which is implemented
|
||||
* by copying nextLocalTransactionId as seen above.
|
||||
*/
|
||||
LocalTransactionId
|
||||
|
||||
@@ -137,8 +137,8 @@ InitRecoveryTransactionEnvironment(void)
|
||||
* are held by vxids and row level locks are held by xids. All queries
|
||||
* hold AccessShareLocks so never block while we write or lock new rows.
|
||||
*/
|
||||
MyProc->vxid.backendId = MyBackendId;
|
||||
vxid.backendId = MyBackendId;
|
||||
MyProc->vxid.procNumber = MyProcNumber;
|
||||
vxid.procNumber = MyProcNumber;
|
||||
vxid.localTransactionId = GetNextLocalTransactionId();
|
||||
VirtualXactLockTableInsert(vxid);
|
||||
|
||||
@@ -300,7 +300,7 @@ LogRecoveryConflict(ProcSignalReason reason, TimestampTz wait_start,
|
||||
vxids = wait_list;
|
||||
while (VirtualTransactionIdIsValid(*vxids))
|
||||
{
|
||||
PGPROC *proc = BackendIdGetProc(vxids->backendId);
|
||||
PGPROC *proc = ProcNumberGetProc(vxids->procNumber);
|
||||
|
||||
/* proc can be NULL if the target backend is not active */
|
||||
if (proc)
|
||||
|
||||
@@ -947,7 +947,7 @@ WaitForLockersMultiple(List *locktags, LOCKMODE lockmode, bool progress)
|
||||
/* If requested, publish who we're going to wait for. */
|
||||
if (progress)
|
||||
{
|
||||
PGPROC *holder = BackendIdGetProc(lockholders->backendId);
|
||||
PGPROC *holder = ProcNumberGetProc(lockholders->procNumber);
|
||||
|
||||
if (holder)
|
||||
pgstat_progress_update_param(PROGRESS_WAITFOR_CURRENT_PID,
|
||||
|
||||
@@ -2995,7 +2995,7 @@ GetLockConflicts(const LOCKTAG *locktag, LOCKMODE lockmode, int *countp)
|
||||
* on this lockable object.
|
||||
*/
|
||||
LWLockRelease(partitionLock);
|
||||
vxids[count].backendId = InvalidBackendId;
|
||||
vxids[count].procNumber = INVALID_PROC_NUMBER;
|
||||
vxids[count].localTransactionId = InvalidLocalTransactionId;
|
||||
if (countp)
|
||||
*countp = count;
|
||||
@@ -3041,7 +3041,7 @@ GetLockConflicts(const LOCKTAG *locktag, LOCKMODE lockmode, int *countp)
|
||||
if (count > MaxBackends + max_prepared_xacts) /* should never happen */
|
||||
elog(PANIC, "too many conflicting locks found");
|
||||
|
||||
vxids[count].backendId = InvalidBackendId;
|
||||
vxids[count].procNumber = INVALID_PROC_NUMBER;
|
||||
vxids[count].localTransactionId = InvalidLocalTransactionId;
|
||||
if (countp)
|
||||
*countp = count;
|
||||
@@ -3625,7 +3625,7 @@ GetLockStatusData(void)
|
||||
proc->fpRelId[f]);
|
||||
instance->holdMask = lockbits << FAST_PATH_LOCKNUMBER_OFFSET;
|
||||
instance->waitLockMode = NoLock;
|
||||
instance->vxid.backendId = proc->vxid.backendId;
|
||||
instance->vxid.procNumber = proc->vxid.procNumber;
|
||||
instance->vxid.localTransactionId = proc->vxid.lxid;
|
||||
instance->pid = proc->pid;
|
||||
instance->leaderPid = proc->pid;
|
||||
@@ -3652,14 +3652,14 @@ GetLockStatusData(void)
|
||||
repalloc(data->locks, sizeof(LockInstanceData) * els);
|
||||
}
|
||||
|
||||
vxid.backendId = proc->vxid.backendId;
|
||||
vxid.procNumber = proc->vxid.procNumber;
|
||||
vxid.localTransactionId = proc->fpLocalTransactionId;
|
||||
|
||||
instance = &data->locks[el];
|
||||
SET_LOCKTAG_VIRTUALTRANSACTION(instance->locktag, vxid);
|
||||
instance->holdMask = LOCKBIT_ON(ExclusiveLock);
|
||||
instance->waitLockMode = NoLock;
|
||||
instance->vxid.backendId = proc->vxid.backendId;
|
||||
instance->vxid.procNumber = proc->vxid.procNumber;
|
||||
instance->vxid.localTransactionId = proc->vxid.lxid;
|
||||
instance->pid = proc->pid;
|
||||
instance->leaderPid = proc->pid;
|
||||
@@ -3712,7 +3712,7 @@ GetLockStatusData(void)
|
||||
instance->waitLockMode = proc->waitLockMode;
|
||||
else
|
||||
instance->waitLockMode = NoLock;
|
||||
instance->vxid.backendId = proc->vxid.backendId;
|
||||
instance->vxid.procNumber = proc->vxid.procNumber;
|
||||
instance->vxid.localTransactionId = proc->vxid.lxid;
|
||||
instance->pid = proc->pid;
|
||||
instance->leaderPid = proclock->groupLeader->pid;
|
||||
@@ -3888,7 +3888,7 @@ GetSingleProcBlockerStatusData(PGPROC *blocked_proc, BlockedProcsData *data)
|
||||
instance->waitLockMode = proc->waitLockMode;
|
||||
else
|
||||
instance->waitLockMode = NoLock;
|
||||
instance->vxid.backendId = proc->vxid.backendId;
|
||||
instance->vxid.procNumber = proc->vxid.procNumber;
|
||||
instance->vxid.localTransactionId = proc->vxid.lxid;
|
||||
instance->pid = proc->pid;
|
||||
instance->leaderPid = proclock->groupLeader->pid;
|
||||
@@ -4391,7 +4391,7 @@ VirtualXactLockTableInsert(VirtualTransactionId vxid)
|
||||
|
||||
LWLockAcquire(&MyProc->fpInfoLock, LW_EXCLUSIVE);
|
||||
|
||||
Assert(MyProc->vxid.backendId == vxid.backendId);
|
||||
Assert(MyProc->vxid.procNumber == vxid.procNumber);
|
||||
Assert(MyProc->fpLocalTransactionId == InvalidLocalTransactionId);
|
||||
Assert(MyProc->fpVXIDLock == false);
|
||||
|
||||
@@ -4413,7 +4413,7 @@ VirtualXactLockTableCleanup(void)
|
||||
bool fastpath;
|
||||
LocalTransactionId lxid;
|
||||
|
||||
Assert(MyProc->vxid.backendId != InvalidBackendId);
|
||||
Assert(MyProc->vxid.procNumber != INVALID_PROC_NUMBER);
|
||||
|
||||
/*
|
||||
* Clean up shared memory state.
|
||||
@@ -4436,7 +4436,7 @@ VirtualXactLockTableCleanup(void)
|
||||
VirtualTransactionId vxid;
|
||||
LOCKTAG locktag;
|
||||
|
||||
vxid.backendId = MyBackendId;
|
||||
vxid.procNumber = MyProcNumber;
|
||||
vxid.localTransactionId = lxid;
|
||||
SET_LOCKTAG_VIRTUALTRANSACTION(locktag, vxid);
|
||||
|
||||
@@ -4530,18 +4530,18 @@ VirtualXactLock(VirtualTransactionId vxid, bool wait)
|
||||
* relevant lxid is no longer running here, that's enough to prove that
|
||||
* it's no longer running anywhere.
|
||||
*/
|
||||
proc = BackendIdGetProc(vxid.backendId);
|
||||
proc = ProcNumberGetProc(vxid.procNumber);
|
||||
if (proc == NULL)
|
||||
return XactLockForVirtualXact(vxid, InvalidTransactionId, wait);
|
||||
|
||||
/*
|
||||
* We must acquire this lock before checking the backendId and lxid
|
||||
* We must acquire this lock before checking the procNumber and lxid
|
||||
* against the ones we're waiting for. The target backend will only set
|
||||
* or clear lxid while holding this lock.
|
||||
*/
|
||||
LWLockAcquire(&proc->fpInfoLock, LW_EXCLUSIVE);
|
||||
|
||||
if (proc->vxid.backendId != vxid.backendId
|
||||
if (proc->vxid.procNumber != vxid.procNumber
|
||||
|| proc->fpLocalTransactionId != vxid.localTransactionId)
|
||||
{
|
||||
/* VXID ended */
|
||||
|
||||
@@ -1260,7 +1260,7 @@ InitPredicateLocks(void)
|
||||
PredXact->OldCommittedSxact->xmin = InvalidTransactionId;
|
||||
PredXact->OldCommittedSxact->flags = SXACT_FLAG_COMMITTED;
|
||||
PredXact->OldCommittedSxact->pid = 0;
|
||||
PredXact->OldCommittedSxact->pgprocno = INVALID_PGPROCNO;
|
||||
PredXact->OldCommittedSxact->pgprocno = INVALID_PROC_NUMBER;
|
||||
}
|
||||
/* This never changes, so let's keep a local copy. */
|
||||
OldCommittedSxact = PredXact->OldCommittedSxact;
|
||||
@@ -4846,7 +4846,7 @@ PostPrepare_PredicateLocks(TransactionId xid)
|
||||
Assert(SxactIsPrepared(MySerializableXact));
|
||||
|
||||
MySerializableXact->pid = 0;
|
||||
MySerializableXact->pgprocno = INVALID_PGPROCNO;
|
||||
MySerializableXact->pgprocno = INVALID_PROC_NUMBER;
|
||||
|
||||
hash_destroy(LocalPredicateLockHash);
|
||||
LocalPredicateLockHash = NULL;
|
||||
@@ -4918,11 +4918,11 @@ predicatelock_twophase_recover(TransactionId xid, uint16 info,
|
||||
(errcode(ERRCODE_OUT_OF_MEMORY),
|
||||
errmsg("out of shared memory")));
|
||||
|
||||
/* vxid for a prepared xact is InvalidBackendId/xid; no pid */
|
||||
sxact->vxid.backendId = InvalidBackendId;
|
||||
/* vxid for a prepared xact is INVALID_PROC_NUMBER/xid; no pid */
|
||||
sxact->vxid.procNumber = INVALID_PROC_NUMBER;
|
||||
sxact->vxid.localTransactionId = (LocalTransactionId) xid;
|
||||
sxact->pid = 0;
|
||||
sxact->pgprocno = INVALID_PGPROCNO;
|
||||
sxact->pgprocno = INVALID_PROC_NUMBER;
|
||||
|
||||
/* a prepared xact hasn't committed yet */
|
||||
sxact->prepareSeqNo = RecoverySerCommitSeqNo;
|
||||
|
||||
@@ -66,7 +66,6 @@ bool log_lock_waits = false;
|
||||
|
||||
/* Pointer to this process's PGPROC struct, if any */
|
||||
PGPROC *MyProc = NULL;
|
||||
int MyProcNumber = INVALID_PGPROCNO;
|
||||
|
||||
/*
|
||||
* This spinlock protects the freelist of recycled PGPROC structures.
|
||||
@@ -181,8 +180,8 @@ InitProcGlobal(void)
|
||||
ProcGlobal->startupBufferPinWaitBufId = -1;
|
||||
ProcGlobal->walwriterLatch = NULL;
|
||||
ProcGlobal->checkpointerLatch = NULL;
|
||||
pg_atomic_init_u32(&ProcGlobal->procArrayGroupFirst, INVALID_PGPROCNO);
|
||||
pg_atomic_init_u32(&ProcGlobal->clogGroupFirst, INVALID_PGPROCNO);
|
||||
pg_atomic_init_u32(&ProcGlobal->procArrayGroupFirst, INVALID_PROC_NUMBER);
|
||||
pg_atomic_init_u32(&ProcGlobal->clogGroupFirst, INVALID_PROC_NUMBER);
|
||||
|
||||
/*
|
||||
* Create and initialize all the PGPROC structures we'll need. There are
|
||||
@@ -275,8 +274,8 @@ InitProcGlobal(void)
|
||||
* Initialize the atomic variables, otherwise, it won't be safe to
|
||||
* access them for backends that aren't currently in use.
|
||||
*/
|
||||
pg_atomic_init_u32(&(proc->procArrayGroupNext), INVALID_PGPROCNO);
|
||||
pg_atomic_init_u32(&(proc->clogGroupNext), INVALID_PGPROCNO);
|
||||
pg_atomic_init_u32(&(proc->procArrayGroupNext), INVALID_PROC_NUMBER);
|
||||
pg_atomic_init_u32(&(proc->clogGroupNext), INVALID_PROC_NUMBER);
|
||||
pg_atomic_init_u64(&(proc->waitStart), 0);
|
||||
}
|
||||
|
||||
@@ -355,7 +354,6 @@ InitProcess(void)
|
||||
errmsg("sorry, too many clients already")));
|
||||
}
|
||||
MyProcNumber = GetNumberFromPGProc(MyProc);
|
||||
MyBackendId = GetBackendIdFromPGProc(MyProc);
|
||||
|
||||
/*
|
||||
* Cross-check that the PGPROC is of the type we expect; if this were not
|
||||
@@ -387,7 +385,7 @@ InitProcess(void)
|
||||
MyProc->xid = InvalidTransactionId;
|
||||
MyProc->xmin = InvalidTransactionId;
|
||||
MyProc->pid = MyProcPid;
|
||||
MyProc->vxid.backendId = MyBackendId;
|
||||
MyProc->vxid.procNumber = MyProcNumber;
|
||||
MyProc->vxid.lxid = InvalidLocalTransactionId;
|
||||
/* databaseId and roleId will be filled in later */
|
||||
MyProc->databaseId = InvalidOid;
|
||||
@@ -423,7 +421,7 @@ InitProcess(void)
|
||||
/* Initialize fields for group XID clearing. */
|
||||
MyProc->procArrayGroupMember = false;
|
||||
MyProc->procArrayGroupMemberXid = InvalidTransactionId;
|
||||
Assert(pg_atomic_read_u32(&MyProc->procArrayGroupNext) == INVALID_PGPROCNO);
|
||||
Assert(pg_atomic_read_u32(&MyProc->procArrayGroupNext) == INVALID_PROC_NUMBER);
|
||||
|
||||
/* Check that group locking fields are in a proper initial state. */
|
||||
Assert(MyProc->lockGroupLeader == NULL);
|
||||
@@ -438,7 +436,7 @@ InitProcess(void)
|
||||
MyProc->clogGroupMemberXidStatus = TRANSACTION_STATUS_IN_PROGRESS;
|
||||
MyProc->clogGroupMemberPage = -1;
|
||||
MyProc->clogGroupMemberLsn = InvalidXLogRecPtr;
|
||||
Assert(pg_atomic_read_u32(&MyProc->clogGroupNext) == INVALID_PGPROCNO);
|
||||
Assert(pg_atomic_read_u32(&MyProc->clogGroupNext) == INVALID_PROC_NUMBER);
|
||||
|
||||
/*
|
||||
* Acquire ownership of the PGPROC's latch, so that we can use WaitLatch
|
||||
@@ -573,7 +571,6 @@ InitAuxiliaryProcess(void)
|
||||
|
||||
MyProc = auxproc;
|
||||
MyProcNumber = GetNumberFromPGProc(MyProc);
|
||||
MyBackendId = GetBackendIdFromPGProc(MyProc);
|
||||
|
||||
/*
|
||||
* Initialize all fields of MyProc, except for those previously
|
||||
@@ -585,7 +582,7 @@ InitAuxiliaryProcess(void)
|
||||
MyProc->fpLocalTransactionId = InvalidLocalTransactionId;
|
||||
MyProc->xid = InvalidTransactionId;
|
||||
MyProc->xmin = InvalidTransactionId;
|
||||
MyProc->vxid.backendId = InvalidBackendId;
|
||||
MyProc->vxid.procNumber = INVALID_PROC_NUMBER;
|
||||
MyProc->vxid.lxid = InvalidLocalTransactionId;
|
||||
MyProc->databaseId = InvalidOid;
|
||||
MyProc->roleId = InvalidOid;
|
||||
@@ -916,13 +913,12 @@ ProcKill(int code, Datum arg)
|
||||
|
||||
proc = MyProc;
|
||||
MyProc = NULL;
|
||||
MyProcNumber = INVALID_PGPROCNO;
|
||||
MyBackendId = InvalidBackendId;
|
||||
MyProcNumber = INVALID_PROC_NUMBER;
|
||||
DisownLatch(&proc->procLatch);
|
||||
|
||||
/* Mark the proc no longer in use */
|
||||
proc->pid = 0;
|
||||
proc->vxid.backendId = InvalidBackendId;
|
||||
proc->vxid.procNumber = INVALID_PROC_NUMBER;
|
||||
proc->vxid.lxid = InvalidTransactionId;
|
||||
|
||||
procgloballist = proc->procgloballist;
|
||||
@@ -998,15 +994,14 @@ AuxiliaryProcKill(int code, Datum arg)
|
||||
|
||||
proc = MyProc;
|
||||
MyProc = NULL;
|
||||
MyProcNumber = INVALID_PGPROCNO;
|
||||
MyBackendId = InvalidBackendId;
|
||||
MyProcNumber = INVALID_PROC_NUMBER;
|
||||
DisownLatch(&proc->procLatch);
|
||||
|
||||
SpinLockAcquire(ProcStructLock);
|
||||
|
||||
/* Mark auxiliary proc no longer in use */
|
||||
proc->pid = 0;
|
||||
proc->vxid.backendId = InvalidBackendId;
|
||||
proc->vxid.procNumber = INVALID_PROC_NUMBER;
|
||||
proc->vxid.lxid = InvalidTransactionId;
|
||||
|
||||
/* Update shared estimate of spins_per_delay */
|
||||
@@ -1863,15 +1858,15 @@ ProcWaitForSignal(uint32 wait_event_info)
|
||||
}
|
||||
|
||||
/*
|
||||
* ProcSendSignal - set the latch of a backend identified by pgprocno
|
||||
* ProcSendSignal - set the latch of a backend identified by ProcNumber
|
||||
*/
|
||||
void
|
||||
ProcSendSignal(int pgprocno)
|
||||
ProcSendSignal(ProcNumber procNumber)
|
||||
{
|
||||
if (pgprocno < 0 || pgprocno >= ProcGlobal->allProcCount)
|
||||
elog(ERROR, "pgprocno out of range");
|
||||
if (procNumber < 0 || procNumber >= ProcGlobal->allProcCount)
|
||||
elog(ERROR, "procNumber out of range");
|
||||
|
||||
SetLatch(&ProcGlobal->allProcs[pgprocno].procLatch);
|
||||
SetLatch(&ProcGlobal->allProcs[procNumber].procLatch);
|
||||
}
|
||||
|
||||
/*
|
||||
|
||||
@@ -1454,7 +1454,7 @@ DropRelationFiles(RelFileLocator *delrels, int ndelrels, bool isRedo)
|
||||
srels = palloc(sizeof(SMgrRelation) * ndelrels);
|
||||
for (i = 0; i < ndelrels; i++)
|
||||
{
|
||||
SMgrRelation srel = smgropen(delrels[i], InvalidBackendId);
|
||||
SMgrRelation srel = smgropen(delrels[i], INVALID_PROC_NUMBER);
|
||||
|
||||
if (isRedo)
|
||||
{
|
||||
@@ -1738,7 +1738,7 @@ _mdnblocks(SMgrRelation reln, ForkNumber forknum, MdfdVec *seg)
|
||||
int
|
||||
mdsyncfiletag(const FileTag *ftag, char *path)
|
||||
{
|
||||
SMgrRelation reln = smgropen(ftag->rlocator, InvalidBackendId);
|
||||
SMgrRelation reln = smgropen(ftag->rlocator, INVALID_PROC_NUMBER);
|
||||
File file;
|
||||
instr_time io_start;
|
||||
bool need_to_close;
|
||||
|
||||
@@ -196,7 +196,7 @@ smgrshutdown(int code, Datum arg)
|
||||
* This does not attempt to actually open the underlying files.
|
||||
*/
|
||||
SMgrRelation
|
||||
smgropen(RelFileLocator rlocator, BackendId backend)
|
||||
smgropen(RelFileLocator rlocator, ProcNumber backend)
|
||||
{
|
||||
RelFileLocatorBackend brlocator;
|
||||
SMgrRelation reln;
|
||||
|
||||
Reference in New Issue
Block a user