Commit fad153ec authored by Tom Lane's avatar Tom Lane

Rewrite the sinval messaging mechanism to reduce contention and avoid

unnecessary cache resets.  The major changes are:

* When the queue overflows, we only issue a cache reset to the specific
backend or backends that still haven't read the oldest message, rather
than resetting everyone as in the original coding.

* When we observe backend(s) falling well behind, we signal SIGUSR1
to only one backend, the one that is furthest behind and doesn't already
have a signal outstanding for it.  When it finishes catching up, it will
in turn signal SIGUSR1 to the next-furthest-back guy, if there is one that
is far enough behind to justify a signal.  The PMSIGNAL_WAKEN_CHILDREN
mechanism is removed.

* We don't attempt to clean out dead messages after every message-receipt
operation; rather, we do it on the insertion side, and only when the queue
fullness passes certain thresholds.

* Split SInvalLock into SInvalReadLock and SInvalWriteLock so that readers
don't block writers nor vice versa (except during the infrequent queue
cleanout operations).

* Transfer multiple sinval messages for each acquisition of a read or
write lock.
parent 30dc388a
...@@ -37,7 +37,7 @@ ...@@ -37,7 +37,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.558 2008/06/06 22:35:22 alvherre Exp $ * $PostgreSQL: pgsql/src/backend/postmaster/postmaster.c,v 1.559 2008/06/19 21:32:56 tgl Exp $
* *
* NOTES * NOTES
* *
...@@ -3829,16 +3829,6 @@ sigusr1_handler(SIGNAL_ARGS) ...@@ -3829,16 +3829,6 @@ sigusr1_handler(SIGNAL_ARGS)
load_role(); load_role();
} }
if (CheckPostmasterSignal(PMSIGNAL_WAKEN_CHILDREN))
{
/*
* Send SIGUSR1 to all children (triggers CatchupInterruptHandler).
* See storage/ipc/sinval[adt].c for the use of this.
*/
if (Shutdown <= SmartShutdown)
SignalChildren(SIGUSR1);
}
if (CheckPostmasterSignal(PMSIGNAL_WAKEN_ARCHIVER) && if (CheckPostmasterSignal(PMSIGNAL_WAKEN_ARCHIVER) &&
PgArchPID != 0) PgArchPID != 0)
{ {
......
...@@ -8,7 +8,7 @@ ...@@ -8,7 +8,7 @@
* *
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/ipc/sinval.c,v 1.85 2008/03/17 11:50:26 alvherre Exp $ * $PostgreSQL: pgsql/src/backend/storage/ipc/sinval.c,v 1.86 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -17,9 +17,7 @@ ...@@ -17,9 +17,7 @@
#include "access/xact.h" #include "access/xact.h"
#include "commands/async.h" #include "commands/async.h"
#include "miscadmin.h" #include "miscadmin.h"
#include "storage/backendid.h"
#include "storage/ipc.h" #include "storage/ipc.h"
#include "storage/proc.h"
#include "storage/sinvaladt.h" #include "storage/sinvaladt.h"
#include "utils/inval.h" #include "utils/inval.h"
...@@ -27,9 +25,9 @@ ...@@ -27,9 +25,9 @@
/* /*
* Because backends sitting idle will not be reading sinval events, we * Because backends sitting idle will not be reading sinval events, we
* need a way to give an idle backend a swift kick in the rear and make * need a way to give an idle backend a swift kick in the rear and make
* it catch up before the sinval queue overflows and forces everyone * it catch up before the sinval queue overflows and forces it to go
* through a cache reset exercise. This is done by broadcasting SIGUSR1 * through a cache reset exercise. This is done by sending SIGUSR1
* to all backends when the queue is threatening to become full. * to any backend that gets too far behind.
* *
* State for catchup events consists of two flags: one saying whether * State for catchup events consists of two flags: one saying whether
* the signal handler is currently allowed to call ProcessCatchupEvent * the signal handler is currently allowed to call ProcessCatchupEvent
...@@ -47,67 +45,101 @@ static void ProcessCatchupEvent(void); ...@@ -47,67 +45,101 @@ static void ProcessCatchupEvent(void);
/* /*
* SendSharedInvalidMessage * SendSharedInvalidMessages
* Add a shared-cache-invalidation message to the global SI message queue. * Add shared-cache-invalidation message(s) to the global SI message queue.
*/ */
void void
SendSharedInvalidMessage(SharedInvalidationMessage *msg) SendSharedInvalidMessages(const SharedInvalidationMessage *msgs, int n)
{ {
bool insertOK; SIInsertDataEntries(msgs, n);
insertOK = SIInsertDataEntry(msg);
if (!insertOK)
elog(DEBUG4, "SI buffer overflow");
} }
/* /*
* ReceiveSharedInvalidMessages * ReceiveSharedInvalidMessages
* Process shared-cache-invalidation messages waiting for this backend * Process shared-cache-invalidation messages waiting for this backend
* *
* We guarantee to process all messages that had been queued before the
* routine was entered. It is of course possible for more messages to get
* queued right after our last SIGetDataEntries call.
*
* NOTE: it is entirely possible for this routine to be invoked recursively * NOTE: it is entirely possible for this routine to be invoked recursively
* as a consequence of processing inside the invalFunction or resetFunction. * as a consequence of processing inside the invalFunction or resetFunction.
* Hence, we must be holding no SI resources when we call them. The only * Furthermore, such a recursive call must guarantee that all outstanding
* bad side-effect is that SIDelExpiredDataEntries might be called extra * inval messages have been processed before it exits. This is the reason
* times on the way out of a nested call. * for the strange-looking choice to use a statically allocated buffer array
* and counters; it's so that a recursive call can process messages already
* sucked out of sinvaladt.c.
*/ */
void void
ReceiveSharedInvalidMessages( ReceiveSharedInvalidMessages(
void (*invalFunction) (SharedInvalidationMessage *msg), void (*invalFunction) (SharedInvalidationMessage *msg),
void (*resetFunction) (void)) void (*resetFunction) (void))
{ {
SharedInvalidationMessage data; #define MAXINVALMSGS 32
int getResult; static SharedInvalidationMessage messages[MAXINVALMSGS];
bool gotMessage = false;
for (;;)
{
/* /*
* We can discard any pending catchup event, since we will not exit * We use volatile here to prevent bugs if a compiler doesn't realize
* this loop until we're fully caught up. * that recursion is a possibility ...
*/ */
catchupInterruptOccurred = 0; static volatile int nextmsg = 0;
static volatile int nummsgs = 0;
/* Deal with any messages still pending from an outer recursion */
while (nextmsg < nummsgs)
{
SharedInvalidationMessage *msg = &messages[nextmsg++];
getResult = SIGetDataEntry(MyBackendId, &data); invalFunction(msg);
}
do
{
int getResult;
nextmsg = nummsgs = 0;
/* Try to get some more messages */
getResult = SIGetDataEntries(messages, MAXINVALMSGS);
if (getResult == 0)
break; /* nothing more to do */
if (getResult < 0) if (getResult < 0)
{ {
/* got a reset message */ /* got a reset message */
elog(DEBUG4, "cache state reset"); elog(DEBUG4, "cache state reset");
resetFunction(); resetFunction();
break; /* nothing more to do */
} }
else
/* Process them, being wary that a recursive call might eat some */
nextmsg = 0;
nummsgs = getResult;
while (nextmsg < nummsgs)
{ {
/* got a normal data message */ SharedInvalidationMessage *msg = &messages[nextmsg++];
invalFunction(&data);
} invalFunction(msg);
gotMessage = true;
} }
/* If we got any messages, try to release dead messages */ /*
if (gotMessage) * We only need to loop if the last SIGetDataEntries call (which
SIDelExpiredDataEntries(false); * might have been within a recursive call) returned a full buffer.
*/
} while (nummsgs == MAXINVALMSGS);
/*
* We are now caught up. If we received a catchup signal, reset that
* flag, and call SICleanupQueue(). This is not so much because we
* need to flush dead messages right now, as that we want to pass on
* the catchup signal to the next slowest backend. "Daisy chaining" the
* catchup signal this way avoids creating spikes in system load for
* what should be just a background maintenance activity.
*/
if (catchupInterruptOccurred)
{
catchupInterruptOccurred = 0;
elog(DEBUG4, "sinval catchup complete, cleaning queue");
SICleanupQueue(false, 0);
}
} }
......
This diff is collapsed.
...@@ -80,7 +80,7 @@ ...@@ -80,7 +80,7 @@
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* IDENTIFICATION * IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/cache/inval.c,v 1.85 2008/06/19 00:46:05 alvherre Exp $ * $PostgreSQL: pgsql/src/backend/utils/cache/inval.c,v 1.86 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -203,7 +203,7 @@ AddInvalidationMessage(InvalidationChunk **listHdr, ...@@ -203,7 +203,7 @@ AddInvalidationMessage(InvalidationChunk **listHdr,
if (chunk == NULL) if (chunk == NULL)
{ {
/* First time through; create initial chunk */ /* First time through; create initial chunk */
#define FIRSTCHUNKSIZE 16 #define FIRSTCHUNKSIZE 32
chunk = (InvalidationChunk *) chunk = (InvalidationChunk *)
MemoryContextAlloc(CurTransactionContext, MemoryContextAlloc(CurTransactionContext,
sizeof(InvalidationChunk) + sizeof(InvalidationChunk) +
...@@ -275,6 +275,23 @@ AppendInvalidationMessageList(InvalidationChunk **destHdr, ...@@ -275,6 +275,23 @@ AppendInvalidationMessageList(InvalidationChunk **destHdr,
} \ } \
} while (0) } while (0)
/*
* Process a list of invalidation messages group-wise.
*
* As above, but the code fragment can handle an array of messages.
* The fragment should refer to the messages as msgs[], with n entries.
*/
#define ProcessMessageListMulti(listHdr, codeFragment) \
do { \
InvalidationChunk *_chunk; \
for (_chunk = (listHdr); _chunk != NULL; _chunk = _chunk->next) \
{ \
SharedInvalidationMessage *msgs = _chunk->msgs; \
int n = _chunk->nitems; \
codeFragment; \
} \
} while (0)
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
* Invalidation set support functions * Invalidation set support functions
...@@ -371,6 +388,18 @@ ProcessInvalidationMessages(InvalidationListHeader *hdr, ...@@ -371,6 +388,18 @@ ProcessInvalidationMessages(InvalidationListHeader *hdr,
ProcessMessageList(hdr->rclist, func(msg)); ProcessMessageList(hdr->rclist, func(msg));
} }
/*
* As above, but the function is able to process an array of messages
* rather than just one at a time.
*/
static void
ProcessInvalidationMessagesMulti(InvalidationListHeader *hdr,
void (*func) (const SharedInvalidationMessage *msgs, int n))
{
ProcessMessageListMulti(hdr->cclist, func(msgs, n));
ProcessMessageListMulti(hdr->rclist, func(msgs, n));
}
/* ---------------------------------------------------------------- /* ----------------------------------------------------------------
* private support functions * private support functions
* ---------------------------------------------------------------- * ----------------------------------------------------------------
...@@ -792,7 +821,7 @@ inval_twophase_postcommit(TransactionId xid, uint16 info, ...@@ -792,7 +821,7 @@ inval_twophase_postcommit(TransactionId xid, uint16 info,
case TWOPHASE_INFO_MSG: case TWOPHASE_INFO_MSG:
msg = (SharedInvalidationMessage *) recdata; msg = (SharedInvalidationMessage *) recdata;
Assert(len == sizeof(SharedInvalidationMessage)); Assert(len == sizeof(SharedInvalidationMessage));
SendSharedInvalidMessage(msg); SendSharedInvalidMessages(msg, 1);
break; break;
case TWOPHASE_INFO_FILE_BEFORE: case TWOPHASE_INFO_FILE_BEFORE:
RelationCacheInitFileInvalidate(true); RelationCacheInitFileInvalidate(true);
...@@ -850,8 +879,8 @@ AtEOXact_Inval(bool isCommit) ...@@ -850,8 +879,8 @@ AtEOXact_Inval(bool isCommit)
AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs, AppendInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs,
&transInvalInfo->CurrentCmdInvalidMsgs); &transInvalInfo->CurrentCmdInvalidMsgs);
ProcessInvalidationMessages(&transInvalInfo->PriorCmdInvalidMsgs, ProcessInvalidationMessagesMulti(&transInvalInfo->PriorCmdInvalidMsgs,
SendSharedInvalidMessage); SendSharedInvalidMessages);
if (transInvalInfo->RelcacheInitFileInval) if (transInvalInfo->RelcacheInitFileInval)
RelationCacheInitFileInvalidate(false); RelationCacheInitFileInvalidate(false);
...@@ -1033,8 +1062,8 @@ EndNonTransactionalInvalidation(void) ...@@ -1033,8 +1062,8 @@ EndNonTransactionalInvalidation(void)
/* Send out the invals */ /* Send out the invals */
ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs, ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs,
LocalExecuteInvalidationMessage); LocalExecuteInvalidationMessage);
ProcessInvalidationMessages(&transInvalInfo->CurrentCmdInvalidMsgs, ProcessInvalidationMessagesMulti(&transInvalInfo->CurrentCmdInvalidMsgs,
SendSharedInvalidMessage); SendSharedInvalidMessages);
/* Clean up and release memory */ /* Clean up and release memory */
for (chunk = transInvalInfo->CurrentCmdInvalidMsgs.cclist; for (chunk = transInvalInfo->CurrentCmdInvalidMsgs.cclist;
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.38 2008/01/01 19:45:59 momjian Exp $ * $PostgreSQL: pgsql/src/include/storage/lwlock.h,v 1.39 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -43,7 +43,8 @@ typedef enum LWLockId ...@@ -43,7 +43,8 @@ typedef enum LWLockId
OidGenLock, OidGenLock,
XidGenLock, XidGenLock,
ProcArrayLock, ProcArrayLock,
SInvalLock, SInvalReadLock,
SInvalWriteLock,
FreeSpaceLock, FreeSpaceLock,
WALInsertLock, WALInsertLock,
WALWriteLock, WALWriteLock,
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/pmsignal.h,v 1.19 2008/01/01 19:45:59 momjian Exp $ * $PostgreSQL: pgsql/src/include/storage/pmsignal.h,v 1.20 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -23,7 +23,6 @@ ...@@ -23,7 +23,6 @@
typedef enum typedef enum
{ {
PMSIGNAL_PASSWORD_CHANGE, /* pg_auth file has changed */ PMSIGNAL_PASSWORD_CHANGE, /* pg_auth file has changed */
PMSIGNAL_WAKEN_CHILDREN, /* send a SIGUSR1 signal to all backends */
PMSIGNAL_WAKEN_ARCHIVER, /* send a NOTIFY signal to xlog archiver */ PMSIGNAL_WAKEN_ARCHIVER, /* send a NOTIFY signal to xlog archiver */
PMSIGNAL_ROTATE_LOGFILE, /* send SIGUSR1 to syslogger to rotate logfile */ PMSIGNAL_ROTATE_LOGFILE, /* send SIGUSR1 to syslogger to rotate logfile */
PMSIGNAL_START_AUTOVAC_LAUNCHER, /* start an autovacuum launcher */ PMSIGNAL_START_AUTOVAC_LAUNCHER, /* start an autovacuum launcher */
......
...@@ -7,7 +7,7 @@ ...@@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/sinval.h,v 1.47 2008/03/16 19:47:34 alvherre Exp $ * $PostgreSQL: pgsql/src/include/storage/sinval.h,v 1.48 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -83,7 +83,8 @@ typedef union ...@@ -83,7 +83,8 @@ typedef union
} SharedInvalidationMessage; } SharedInvalidationMessage;
extern void SendSharedInvalidMessage(SharedInvalidationMessage *msg); extern void SendSharedInvalidMessages(const SharedInvalidationMessage *msgs,
int n);
extern void ReceiveSharedInvalidMessages( extern void ReceiveSharedInvalidMessages(
void (*invalFunction) (SharedInvalidationMessage *msg), void (*invalFunction) (SharedInvalidationMessage *msg),
void (*resetFunction) (void)); void (*resetFunction) (void));
......
/*------------------------------------------------------------------------- /*-------------------------------------------------------------------------
* *
* sinvaladt.h * sinvaladt.h
* POSTGRES shared cache invalidation segment definitions. * POSTGRES shared cache invalidation data manager.
* *
* The shared cache invalidation manager is responsible for transmitting * The shared cache invalidation manager is responsible for transmitting
* invalidation messages between backends. Any message sent by any backend * invalidation messages between backends. Any message sent by any backend
* must be delivered to all already-running backends before it can be * must be delivered to all already-running backends before it can be
* forgotten. * forgotten. (If we run out of space, we instead deliver a "RESET"
* message to backends that have fallen too far behind.)
* *
* The struct type SharedInvalidationMessage, defining the contents of * The struct type SharedInvalidationMessage, defining the contents of
* a single message, is defined in sinval.h. * a single message, is defined in sinval.h.
...@@ -14,7 +15,7 @@ ...@@ -14,7 +15,7 @@
* Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group * Portions Copyright (c) 1996-2008, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California * Portions Copyright (c) 1994, Regents of the University of California
* *
* $PostgreSQL: pgsql/src/include/storage/sinvaladt.h,v 1.47 2008/03/17 11:50:27 alvherre Exp $ * $PostgreSQL: pgsql/src/include/storage/sinvaladt.h,v 1.48 2008/06/19 21:32:56 tgl Exp $
* *
*------------------------------------------------------------------------- *-------------------------------------------------------------------------
*/ */
...@@ -23,7 +24,6 @@ ...@@ -23,7 +24,6 @@
#include "storage/sinval.h" #include "storage/sinval.h"
/* /*
* prototypes for functions in sinvaladt.c * prototypes for functions in sinvaladt.c
*/ */
...@@ -31,9 +31,9 @@ extern Size SInvalShmemSize(void); ...@@ -31,9 +31,9 @@ extern Size SInvalShmemSize(void);
extern void CreateSharedInvalidationState(void); extern void CreateSharedInvalidationState(void);
extern void SharedInvalBackendInit(void); extern void SharedInvalBackendInit(void);
extern bool SIInsertDataEntry(SharedInvalidationMessage *data); extern void SIInsertDataEntries(const SharedInvalidationMessage *data, int n);
extern int SIGetDataEntry(int backendId, SharedInvalidationMessage *data); extern int SIGetDataEntries(SharedInvalidationMessage *data, int datasize);
extern void SIDelExpiredDataEntries(bool locked); extern void SICleanupQueue(bool callerHasWriteLock, int minFree);
extern LocalTransactionId GetNextLocalTransactionId(void); extern LocalTransactionId GetNextLocalTransactionId(void);
......
Markdown is supported
0% or
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment