postgresql/src/backend/postmaster/checkpointer.c

Ignoring revisions in .git-blame-ignore-revs. Click here to bypass and see the normal blame view.

1343 lines
41 KiB
C
Raw Normal View History

2011-11-01 19:07:29 +01:00
/*-------------------------------------------------------------------------
*
* checkpointer.c
*
* The checkpointer is new as of Postgres 9.2. It handles all checkpoints.
* Checkpoints are automatically dispatched after a certain amount of time has
* elapsed since the last one, and it can be signaled to perform requested
* checkpoints as well. (The GUC parameter that mandates a checkpoint every
* so many WAL segments is implemented by having backends signal when they
* fill WAL segments; the checkpointer itself doesn't watch for the
* condition.)
*
* Normal termination is by SIGUSR2, which instructs the checkpointer to
* execute a shutdown checkpoint and then exit(0). (All backends must be
* stopped before SIGUSR2 is issued!) Emergency termination is by SIGQUIT;
* like any backend, the checkpointer will simply abort and exit on SIGQUIT.
2011-11-01 19:07:29 +01:00
*
* If the checkpointer exits unexpectedly, the postmaster treats that the same
* as a backend crash: shared memory may be corrupted, so remaining backends
* should be killed by SIGQUIT and then a recovery cycle started. (Even if
* shared memory isn't corrupted, we have lost information about which
* files need to be fsync'd for the next checkpoint, and so a system
* restart needs to be forced.)
*
*
* Portions Copyright (c) 1996-2024, PostgreSQL Global Development Group
2011-11-01 19:07:29 +01:00
*
*
* IDENTIFICATION
* src/backend/postmaster/checkpointer.c
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include <sys/time.h>
#include <time.h>
2011-11-01 19:07:29 +01:00
#include "access/xlog.h"
2011-11-01 19:07:29 +01:00
#include "access/xlog_internal.h"
#include "access/xlogrecovery.h"
2011-11-01 19:07:29 +01:00
#include "libpq/pqsignal.h"
#include "miscadmin.h"
#include "pgstat.h"
#include "postmaster/auxprocess.h"
2011-11-01 19:07:29 +01:00
#include "postmaster/bgwriter.h"
#include "postmaster/interrupt.h"
2011-11-01 19:07:29 +01:00
#include "replication/syncrep.h"
#include "storage/bufmgr.h"
#include "storage/condition_variable.h"
#include "storage/fd.h"
2011-11-01 19:07:29 +01:00
#include "storage/ipc.h"
#include "storage/lwlock.h"
#include "storage/proc.h"
#include "storage/procsignal.h"
2011-11-01 19:07:29 +01:00
#include "storage/shmem.h"
#include "storage/smgr.h"
#include "storage/spin.h"
#include "utils/guc.h"
#include "utils/memutils.h"
#include "utils/resowner.h"
/*----------
* Shared memory area for communication between checkpointer and backends
*
* The ckpt counters allow backends to watch for completion of a checkpoint
* request they send. Here's how it works:
* * At start of a checkpoint, checkpointer reads (and clears) the request
* flags and increments ckpt_started, while holding ckpt_lck.
2011-11-01 19:07:29 +01:00
* * On completion of a checkpoint, checkpointer sets ckpt_done to
* equal ckpt_started.
* * On failure of a checkpoint, checkpointer increments ckpt_failed
* and sets ckpt_done to equal ckpt_started.
*
* The algorithm for backends is:
* 1. Record current values of ckpt_failed and ckpt_started, and
* set request flags, while holding ckpt_lck.
* 2. Send signal to request checkpoint.
* 3. Sleep until ckpt_started changes. Now you know a checkpoint has
* begun since you started this algorithm (although *not* that it was
* specifically initiated by your signal), and that it is using your flags.
* 4. Record new value of ckpt_started.
* 5. Sleep until ckpt_done >= saved value of ckpt_started. (Use modulo
* arithmetic here in case counters wrap around.) Now you know a
* checkpoint has started and completed, but not whether it was
* successful.
* 6. If ckpt_failed is different from the originally saved value,
* assume request failed; otherwise it was definitely successful.
*
* ckpt_flags holds the OR of the checkpoint request flags sent by all
* requesting backends since the last checkpoint start. The flags are
* chosen so that OR'ing is the correct way to combine multiple requests.
*
* The requests array holds fsync requests sent by backends and not yet
* absorbed by the checkpointer.
*
* Unlike the checkpoint fields, requests related fields are protected by
* CheckpointerCommLock.
2011-11-01 19:07:29 +01:00
*----------
*/
typedef struct
{
SyncRequestType type; /* request type */
FileTag ftag; /* file identifier */
} CheckpointerRequest;
2011-11-01 19:07:29 +01:00
typedef struct
{
pid_t checkpointer_pid; /* PID (0 if not started) */
slock_t ckpt_lck; /* protects all the ckpt_* fields */
int ckpt_started; /* advances when checkpoint starts */
int ckpt_done; /* advances when checkpoint done */
int ckpt_failed; /* advances when checkpoint fails */
int ckpt_flags; /* checkpoint flags, as defined in xlog.h */
ConditionVariable start_cv; /* signaled when ckpt_started advances */
ConditionVariable done_cv; /* signaled when ckpt_done advances */
2011-11-01 19:07:29 +01:00
int num_requests; /* current # of requests */
int max_requests; /* allocated array size */
CheckpointerRequest requests[FLEXIBLE_ARRAY_MEMBER];
} CheckpointerShmemStruct;
2011-11-01 19:07:29 +01:00
static CheckpointerShmemStruct *CheckpointerShmem;
2011-11-01 19:07:29 +01:00
/* interval for calling AbsorbSyncRequests in CheckpointWriteDelay */
2011-11-01 19:07:29 +01:00
#define WRITES_PER_ABSORB 1000
/*
* GUC parameters
*/
int CheckPointTimeout = 300;
int CheckPointWarning = 30;
double CheckPointCompletionTarget = 0.9;
2011-11-01 19:07:29 +01:00
/*
* Private state
*/
static bool ckpt_active = false;
/* these values are valid when ckpt_active is true: */
static pg_time_t ckpt_start_time;
static XLogRecPtr ckpt_start_recptr;
static double ckpt_cached_elapsed;
static pg_time_t last_checkpoint_time;
static pg_time_t last_xlog_switch_time;
/* Prototypes for private functions */
static void HandleCheckpointerInterrupts(void);
2011-11-01 19:07:29 +01:00
static void CheckArchiveTimeout(void);
static bool IsCheckpointOnSchedule(double progress);
static bool ImmediateCheckpointRequested(void);
static bool CompactCheckpointerRequestQueue(void);
static void UpdateSharedMemoryConfig(void);
2011-11-01 19:07:29 +01:00
/* Signal handlers */
static void ReqCheckpointHandler(SIGNAL_ARGS);
/*
* Main entry point for checkpointer process
*
Fix management of pendingOpsTable in auxiliary processes. mdinit() was misusing IsBootstrapProcessingMode() to decide whether to create an fsync pending-operations table in the current process. This led to creating a table not only in the startup and checkpointer processes as intended, but also in the bgwriter process, not to mention other auxiliary processes such as walwriter and walreceiver. Creation of the table in the bgwriter is fatal, because it absorbs fsync requests that should have gone to the checkpointer; instead they just sit in bgwriter local memory and are never acted on. So writes performed by the bgwriter were not being fsync'd which could result in data loss after an OS crash. I think there is no live bug with respect to walwriter and walreceiver because those never perform any writes of shared buffers; but the potential is there for future breakage in those processes too. To fix, make AuxiliaryProcessMain() export the current process's AuxProcType as a global variable, and then make mdinit() test directly for the types of aux process that should have a pendingOpsTable. Having done that, we might as well also get rid of the random bool flags such as am_walreceiver that some of the aux processes had grown. (Note that we could not have fixed the bug by examining those variables in mdinit(), because it's called from BaseInit() which is run by AuxiliaryProcessMain() before entering any of the process-type-specific code.) Back-patch to 9.2, where the problem was introduced by the split-up of bgwriter and checkpointer processes. The bogus pendingOpsTable exists in walwriter and walreceiver processes in earlier branches, but absent any evidence that it causes actual problems there, I'll leave the older branches alone.
2012-07-18 21:28:10 +02:00
* This is invoked from AuxiliaryProcessMain, which has already created the
* basic execution environment, but not enabled signals yet.
2011-11-01 19:07:29 +01:00
*/
void
CheckpointerMain(char *startup_data, size_t startup_data_len)
2011-11-01 19:07:29 +01:00
{
sigjmp_buf local_sigjmp_buf;
MemoryContext checkpointer_context;
Assert(startup_data_len == 0);
MyBackendType = B_CHECKPOINTER;
AuxiliaryProcessMainCommon();
CheckpointerShmem->checkpointer_pid = MyProcPid;
2011-11-01 19:07:29 +01:00
/*
* Properly accept or ignore signals the postmaster might send us
*
* Note: we deliberately ignore SIGTERM, because during a standard Unix
* system shutdown cycle, init will SIGTERM all processes at once. We
* want to wait for the backends to exit, whereupon the postmaster will
* tell us it's okay to shut down (via SIGUSR2).
*/
pqsignal(SIGHUP, SignalHandlerForConfigReload);
2011-11-01 19:07:29 +01:00
pqsignal(SIGINT, ReqCheckpointHandler); /* request checkpoint */
pqsignal(SIGTERM, SIG_IGN); /* ignore SIGTERM */
/* SIGQUIT handler was already set up by InitPostmasterChild */
2011-11-01 19:07:29 +01:00
pqsignal(SIGALRM, SIG_IGN);
pqsignal(SIGPIPE, SIG_IGN);
pqsignal(SIGUSR1, procsignal_sigusr1_handler);
pqsignal(SIGUSR2, SignalHandlerForShutdownRequest);
2011-11-01 19:07:29 +01:00
/*
* Reset some signals that are accepted by postmaster but not here
*/
pqsignal(SIGCHLD, SIG_DFL);
/*
* Initialize so that first time-driven event happens at the correct time.
*/
last_checkpoint_time = last_xlog_switch_time = (pg_time_t) time(NULL);
pgstat: store statistics in shared memory. Previously the statistics collector received statistics updates via UDP and shared statistics data by writing them out to temporary files regularly. These files can reach tens of megabytes and are written out up to twice a second. This has repeatedly prevented us from adding additional useful statistics. Now statistics are stored in shared memory. Statistics for variable-numbered objects are stored in a dshash hashtable (backed by dynamic shared memory). Fixed-numbered stats are stored in plain shared memory. The header for pgstat.c contains an overview of the architecture. The stats collector is not needed anymore, remove it. By utilizing the transactional statistics drop infrastructure introduced in a prior commit statistics entries cannot "leak" anymore. Previously leaked statistics were dropped by pgstat_vacuum_stat(), called from [auto-]vacuum. On systems with many small relations pgstat_vacuum_stat() could be quite expensive. Now that replicas drop statistics entries for dropped objects, it is not necessary anymore to reset stats when starting from a cleanly shut down replica. Subsequent commits will perform some further code cleanup, adapt docs and add tests. Bumps PGSTAT_FILE_FORMAT_ID. Author: Kyotaro Horiguchi <horikyota.ntt@gmail.com> Author: Andres Freund <andres@anarazel.de> Author: Melanie Plageman <melanieplageman@gmail.com> Reviewed-By: Andres Freund <andres@anarazel.de> Reviewed-By: Thomas Munro <thomas.munro@gmail.com> Reviewed-By: Justin Pryzby <pryzby@telsasoft.com> Reviewed-By: "David G. Johnston" <david.g.johnston@gmail.com> Reviewed-By: Tomas Vondra <tomas.vondra@2ndquadrant.com> (in a much earlier version) Reviewed-By: Arthur Zakirov <a.zakirov@postgrespro.ru> (in a much earlier version) Reviewed-By: Antonin Houska <ah@cybertec.at> (in a much earlier version) Discussion: https://postgr.es/m/20220303021600.hs34ghqcw6zcokdh@alap3.anarazel.de Discussion: https://postgr.es/m/20220308205351.2xcn6k4x5yivcxyd@alap3.anarazel.de Discussion: https://postgr.es/m/20210319235115.y3wz7hpnnrshdyv6@alap3.anarazel.de
2022-04-07 06:29:46 +02:00
/*
* Write out stats after shutdown. This needs to be called by exactly one
* process during a normal shutdown, and since checkpointer is shut down
* very late...
*
* Walsenders are shut down after the checkpointer, but currently don't
* report stats. If that changes, we need a more complicated solution.
*/
before_shmem_exit(pgstat_before_server_shutdown, 0);
2011-11-01 19:07:29 +01:00
/*
* Create a memory context that we will do all our work in. We do this so
* that we can reset the context during error recovery and thereby avoid
* possible memory leaks. Formerly this code just ran in
* TopMemoryContext, but resetting that would be a really bad idea.
*/
checkpointer_context = AllocSetContextCreate(TopMemoryContext,
"Checkpointer",
Add macros to make AllocSetContextCreate() calls simpler and safer. I found that half a dozen (nearly 5%) of our AllocSetContextCreate calls had typos in the context-sizing parameters. While none of these led to especially significant problems, they did create minor inefficiencies, and it's now clear that expecting people to copy-and-paste those calls accurately is not a great idea. Let's reduce the risk of future errors by introducing single macros that encapsulate the common use-cases. Three such macros are enough to cover all but two special-purpose contexts; those two calls can be left as-is, I think. While this patch doesn't in itself improve matters for third-party extensions, it doesn't break anything for them either, and they can gradually adopt the simplified notation over time. In passing, change TopMemoryContext to use the default allocation parameters. Formerly it could only be extended 8K at a time. That was probably reasonable when this code was written; but nowadays we create many more contexts than we did then, so that it's not unusual to have a couple hundred K in TopMemoryContext, even without considering various dubious code that sticks other things there. There seems no good reason not to let it use growing blocks like most other contexts. Back-patch to 9.6, mostly because that's still close enough to HEAD that it's easy to do so, and keeping the branches in sync can be expected to avoid some future back-patching pain. The bugs fixed by these changes don't seem to be significant enough to justify fixing them further back. Discussion: <21072.1472321324@sss.pgh.pa.us>
2016-08-27 23:50:38 +02:00
ALLOCSET_DEFAULT_SIZES);
2011-11-01 19:07:29 +01:00
MemoryContextSwitchTo(checkpointer_context);
/*
* If an exception is encountered, processing resumes here.
*
* You might wonder why this isn't coded as an infinite loop around a
* PG_TRY construct. The reason is that this is the bottom of the
* exception stack, and so with PG_TRY there would be no exception handler
* in force at all during the CATCH part. By leaving the outermost setjmp
* always active, we have at least some chance of recovering from an error
* during error recovery. (If we get into an infinite loop thereby, it
* will soon be stopped by overflow of elog.c's internal state stack.)
*
* Note that we use sigsetjmp(..., 1), so that the prevailing signal mask
* (to wit, BlockSig) will be restored when longjmp'ing to here. Thus,
* signals other than SIGQUIT will be blocked until we complete error
* recovery. It might seem that this policy makes the HOLD_INTERRUPTS()
* call redundant, but it is not since InterruptPending might be set
* already.
2011-11-01 19:07:29 +01:00
*/
if (sigsetjmp(local_sigjmp_buf, 1) != 0)
{
/* Since not using PG_TRY, must reset error stack by hand */
error_context_stack = NULL;
/* Prevent interrupts while cleaning up */
HOLD_INTERRUPTS();
/* Report the error to the server log */
EmitErrorReport();
/*
* These operations are really just a minimal subset of
* AbortTransaction(). We don't have very many resources to worry
* about in checkpointer, but we do have LWLocks, buffers, and temp
* files.
*/
LWLockReleaseAll();
ConditionVariableCancelSleep();
pgstat_report_wait_end();
2011-11-01 19:07:29 +01:00
UnlockBuffers();
Use a ResourceOwner to track buffer pins in all cases. Historically, we've allowed auxiliary processes to take buffer pins without tracking them in a ResourceOwner. However, that creates problems for error recovery. In particular, we've seen multiple reports of assertion crashes in the startup process when it gets an error while holding a buffer pin, as for example if it gets ENOSPC during a write. In a non-assert build, the process would simply exit without releasing the pin at all. We've gotten away with that so far just because a failure exit of the startup process translates to a database crash anyhow; but any similar behavior in other aux processes could result in stuck pins and subsequent problems in vacuum. To improve this, institute a policy that we must *always* have a resowner backing any attempt to pin a buffer, which we can enforce just by removing the previous special-case code in resowner.c. Add infrastructure to make it easy to create a process-lifespan AuxProcessResourceOwner and clear out its contents at appropriate times. Replace existing ad-hoc resowner management in bgwriter.c and other aux processes with that. (Thus, while the startup process gains a resowner where it had none at all before, some other aux process types are replacing an ad-hoc resowner with this code.) Also use the AuxProcessResourceOwner to manage buffer pins taken during StartupXLOG and ShutdownXLOG, even when those are being run in a bootstrap process or a standalone backend rather than a true auxiliary process. In passing, remove some other ad-hoc resource owner creations that had gotten cargo-culted into various other places. As far as I can tell that was all unnecessary, and if it had been necessary it was incomplete, due to lacking any provision for clearing those resowners later. (Also worth noting in this connection is that a process that hasn't called InitBufferPoolBackend has no business accessing buffers; so there's more to do than just add the resowner if we want to touch buffers in processes not covered by this patch.) Although this fixes a very old bug, no back-patch, because there's no evidence of any significant problem in non-assert builds. Patch by me, pursuant to a report from Justin Pryzby. Thanks to Robert Haas and Kyotaro Horiguchi for reviews. Discussion: https://postgr.es/m/20180627233939.GA10276@telsasoft.com
2018-07-18 18:15:16 +02:00
ReleaseAuxProcessResources(false);
2011-11-01 19:07:29 +01:00
AtEOXact_Buffers(false);
AtEOXact_SMgr();
AtEOXact_Files(false);
2011-11-01 19:07:29 +01:00
AtEOXact_HashTables(false);
/* Warn any waiting backends that the checkpoint failed. */
if (ckpt_active)
{
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
CheckpointerShmem->ckpt_failed++;
CheckpointerShmem->ckpt_done = CheckpointerShmem->ckpt_started;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
ConditionVariableBroadcast(&CheckpointerShmem->done_cv);
2011-11-01 19:07:29 +01:00
ckpt_active = false;
}
/*
* Now return to normal top-level context and clear ErrorContext for
* next time.
*/
MemoryContextSwitchTo(checkpointer_context);
FlushErrorState();
/* Flush any leaked data in the top-level context */
MemoryContextReset(checkpointer_context);
2011-11-01 19:07:29 +01:00
/* Now we can allow interrupts again */
RESUME_INTERRUPTS();
/*
* Sleep at least 1 second after any error. A write error is likely
* to be repeated, and we don't want to be filling the error logs as
* fast as we can.
*/
pg_usleep(1000000L);
}
/* We can now handle ereport(ERROR) */
PG_exception_stack = &local_sigjmp_buf;
/*
* Unblock signals (they were blocked when the postmaster forked us)
*/
sigprocmask(SIG_SETMASK, &UnBlockSig, NULL);
2011-11-01 19:07:29 +01:00
/*
* Ensure all shared memory values are set correctly for the config. Doing
* this here ensures no race conditions from other concurrent updaters.
*/
UpdateSharedMemoryConfig();
2011-11-01 19:07:29 +01:00
/*
* Advertise our latch that backends can use to wake us up while we're
* sleeping.
*/
ProcGlobal->checkpointerLatch = &MyProc->procLatch;
2011-11-01 19:07:29 +01:00
/*
* Loop forever
*/
for (;;)
{
bool do_checkpoint = false;
int flags = 0;
pg_time_t now;
int elapsed_secs;
int cur_timeout;
bool chkpt_or_rstpt_requested = false;
bool chkpt_or_rstpt_timed = false;
/* Clear any already-pending wakeups */
ResetLatch(MyLatch);
2011-11-01 19:07:29 +01:00
/*
* Process any requests or signals received recently.
*/
AbsorbSyncRequests();
HandleCheckpointerInterrupts();
2011-11-01 19:07:29 +01:00
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
/*
* Detect a pending checkpoint request by checking whether the flags
* word in shared memory is nonzero. We shouldn't need to acquire the
* ckpt_lck for this.
*/
if (((volatile CheckpointerShmemStruct *) CheckpointerShmem)->ckpt_flags)
{
do_checkpoint = true;
chkpt_or_rstpt_requested = true;
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
}
2011-11-01 19:07:29 +01:00
/*
* Force a checkpoint if too much time has elapsed since the last one.
* Note that we count a timed checkpoint in stats only when this
* occurs without an external request, but we set the CAUSE_TIME flag
* bit even if there is also an external request.
*/
now = (pg_time_t) time(NULL);
elapsed_secs = now - last_checkpoint_time;
if (elapsed_secs >= CheckPointTimeout)
{
if (!do_checkpoint)
chkpt_or_rstpt_timed = true;
2011-11-01 19:07:29 +01:00
do_checkpoint = true;
flags |= CHECKPOINT_CAUSE_TIME;
}
/*
* Do a checkpoint if requested.
*/
if (do_checkpoint)
{
bool ckpt_performed = false;
bool do_restartpoint;
/* Check if we should perform a checkpoint or a restartpoint. */
2011-11-01 19:07:29 +01:00
do_restartpoint = RecoveryInProgress();
/*
* Atomically fetch the request flags to figure out what kind of a
* checkpoint we should perform, and increase the started-counter
* to acknowledge that we've started a new checkpoint.
*/
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
flags |= CheckpointerShmem->ckpt_flags;
CheckpointerShmem->ckpt_flags = 0;
CheckpointerShmem->ckpt_started++;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
ConditionVariableBroadcast(&CheckpointerShmem->start_cv);
2011-11-01 19:07:29 +01:00
/*
* The end-of-recovery checkpoint is a real checkpoint that's
* performed while we're still in recovery.
*/
if (flags & CHECKPOINT_END_OF_RECOVERY)
do_restartpoint = false;
if (chkpt_or_rstpt_timed)
{
chkpt_or_rstpt_timed = false;
if (do_restartpoint)
PendingCheckpointerStats.restartpoints_timed++;
else
PendingCheckpointerStats.num_timed++;
}
if (chkpt_or_rstpt_requested)
{
chkpt_or_rstpt_requested = false;
if (do_restartpoint)
PendingCheckpointerStats.restartpoints_requested++;
else
PendingCheckpointerStats.num_requested++;
}
2011-11-01 19:07:29 +01:00
/*
* We will warn if (a) too soon since last checkpoint (whatever
* caused it) and (b) somebody set the CHECKPOINT_CAUSE_XLOG flag
* since the last checkpoint start. Note in particular that this
* implementation will not generate warnings caused by
* CheckPointTimeout < CheckPointWarning.
*/
if (!do_restartpoint &&
(flags & CHECKPOINT_CAUSE_XLOG) &&
elapsed_secs < CheckPointWarning)
ereport(LOG,
(errmsg_plural("checkpoints are occurring too frequently (%d second apart)",
"checkpoints are occurring too frequently (%d seconds apart)",
elapsed_secs,
elapsed_secs),
errhint("Consider increasing the configuration parameter max_wal_size.")));
2011-11-01 19:07:29 +01:00
/*
* Initialize checkpointer-private variables used during
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
* checkpoint.
2011-11-01 19:07:29 +01:00
*/
ckpt_active = true;
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
if (do_restartpoint)
ckpt_start_recptr = GetXLogReplayRecPtr(NULL);
else
2011-11-01 19:07:29 +01:00
ckpt_start_recptr = GetInsertRecPtr();
ckpt_start_time = now;
ckpt_cached_elapsed = 0;
/*
* Do the checkpoint.
*/
if (!do_restartpoint)
{
CreateCheckPoint(flags);
ckpt_performed = true;
}
else
ckpt_performed = CreateRestartPoint(flags);
/*
Give SMgrRelation pointers a well-defined lifetime. After calling smgropen(), it was not clear how long you could continue to use the result, because various code paths including cache invalidation could call smgrclose(), which freed the memory. Guarantee that the object won't be destroyed until the end of the current transaction, or in recovery, the commit/abort record that destroys the underlying storage. smgrclose() is now just an alias for smgrrelease(). It closes files and forgets all state except the rlocator, but keeps the SMgrRelation object valid. A new smgrdestroy() function is used by rare places that know there should be no other references to the SMgrRelation. The short version: * smgrclose() is now just an alias for smgrrelease(). It releases resources, but doesn't destroy until EOX * smgrdestroy() now frees memory, and should rarely be used. Existing code should be unaffected, but it is now possible for code that has an SMgrRelation object to use it repeatedly during a transaction as long as the storage hasn't been physically dropped. Such code would normally hold a lock on the relation. This also replaces the "ownership" mechanism of SMgrRelations with a pin counter. An SMgrRelation can now be "pinned", which prevents it from being destroyed at end of transaction. There can be multiple pins on the same SMgrRelation. In practice, the pin mechanism is only used by the relcache, so there cannot be more than one pin on the same SMgrRelation. Except with swap_relation_files XXX Author: Thomas Munro, Heikki Linnakangas Reviewed-by: Robert Haas <robertmhaas@gmail.com> Discussion: https://www.postgresql.org/message-id/CA%2BhUKGJ8NTvqLHz6dqbQnt2c8XCki4r2QvXjBQcXpVwxTY_pvA@mail.gmail.com
2024-01-31 11:31:02 +01:00
* After any checkpoint, free all smgr objects. Otherwise we
* would never do so for dropped relations, as the checkpointer
* does not process shared invalidation messages or call
* AtEOXact_SMgr().
2011-11-01 19:07:29 +01:00
*/
Give SMgrRelation pointers a well-defined lifetime. After calling smgropen(), it was not clear how long you could continue to use the result, because various code paths including cache invalidation could call smgrclose(), which freed the memory. Guarantee that the object won't be destroyed until the end of the current transaction, or in recovery, the commit/abort record that destroys the underlying storage. smgrclose() is now just an alias for smgrrelease(). It closes files and forgets all state except the rlocator, but keeps the SMgrRelation object valid. A new smgrdestroy() function is used by rare places that know there should be no other references to the SMgrRelation. The short version: * smgrclose() is now just an alias for smgrrelease(). It releases resources, but doesn't destroy until EOX * smgrdestroy() now frees memory, and should rarely be used. Existing code should be unaffected, but it is now possible for code that has an SMgrRelation object to use it repeatedly during a transaction as long as the storage hasn't been physically dropped. Such code would normally hold a lock on the relation. This also replaces the "ownership" mechanism of SMgrRelations with a pin counter. An SMgrRelation can now be "pinned", which prevents it from being destroyed at end of transaction. There can be multiple pins on the same SMgrRelation. In practice, the pin mechanism is only used by the relcache, so there cannot be more than one pin on the same SMgrRelation. Except with swap_relation_files XXX Author: Thomas Munro, Heikki Linnakangas Reviewed-by: Robert Haas <robertmhaas@gmail.com> Discussion: https://www.postgresql.org/message-id/CA%2BhUKGJ8NTvqLHz6dqbQnt2c8XCki4r2QvXjBQcXpVwxTY_pvA@mail.gmail.com
2024-01-31 11:31:02 +01:00
smgrdestroyall();
2011-11-01 19:07:29 +01:00
/*
* Indicate checkpoint completion to any waiting backends.
*/
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
CheckpointerShmem->ckpt_done = CheckpointerShmem->ckpt_started;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
ConditionVariableBroadcast(&CheckpointerShmem->done_cv);
2011-11-01 19:07:29 +01:00
if (ckpt_performed)
{
/*
* Note we record the checkpoint start time not end time as
* last_checkpoint_time. This is so that time-driven
* checkpoints happen at a predictable spacing.
*/
last_checkpoint_time = now;
if (do_restartpoint)
PendingCheckpointerStats.restartpoints_performed++;
2011-11-01 19:07:29 +01:00
}
else
{
/*
* We were not able to perform the restartpoint (checkpoints
* throw an ERROR in case of error). Most likely because we
* have not received any new checkpoint WAL records since the
* last restartpoint. Try again in 15 s.
*/
last_checkpoint_time = now - CheckPointTimeout + 15;
}
ckpt_active = false;
/* We may have received an interrupt during the checkpoint. */
HandleCheckpointerInterrupts();
2011-11-01 19:07:29 +01:00
}
/* Check for archive_timeout and switch xlog files if necessary. */
CheckArchiveTimeout();
/* Report pending statistics to the cumulative stats system */
pgstat_report_checkpointer();
pgstat_report_wal(true);
/*
* If any checkpoint flags have been set, redo the loop to handle the
* checkpoint without sleeping.
*/
if (((volatile CheckpointerShmemStruct *) CheckpointerShmem)->ckpt_flags)
continue;
2011-11-01 19:07:29 +01:00
/*
* Sleep until we are signaled or it's time for another checkpoint or
* xlog file switch.
2011-11-01 19:07:29 +01:00
*/
now = (pg_time_t) time(NULL);
elapsed_secs = now - last_checkpoint_time;
if (elapsed_secs >= CheckPointTimeout)
continue; /* no sleep for us ... */
cur_timeout = CheckPointTimeout - elapsed_secs;
if (XLogArchiveTimeout > 0 && !RecoveryInProgress())
{
elapsed_secs = now - last_xlog_switch_time;
if (elapsed_secs >= XLogArchiveTimeout)
continue; /* no sleep for us ... */
cur_timeout = Min(cur_timeout, XLogArchiveTimeout - elapsed_secs);
}
2011-11-01 19:07:29 +01:00
2018-11-23 08:16:41 +01:00
(void) WaitLatch(MyLatch,
WL_LATCH_SET | WL_TIMEOUT | WL_EXIT_ON_PM_DEATH,
cur_timeout * 1000L /* convert to ms */ ,
WAIT_EVENT_CHECKPOINTER_MAIN);
2011-11-01 19:07:29 +01:00
}
}
/*
* Process any new interrupts.
*/
static void
HandleCheckpointerInterrupts(void)
{
if (ProcSignalBarrierPending)
ProcessProcSignalBarrier();
if (ConfigReloadPending)
{
ConfigReloadPending = false;
ProcessConfigFile(PGC_SIGHUP);
/*
* Checkpointer is the last process to shut down, so we ask it to hold
* the keys for a range of other tasks required most of which have
* nothing to do with checkpointing at all.
*
* For various reasons, some config values can change dynamically so
* the primary copy of them is held in shared memory to make sure all
* backends see the same value. We make Checkpointer responsible for
* updating the shared memory copy if the parameter setting changes
* because of SIGHUP.
*/
UpdateSharedMemoryConfig();
}
if (ShutdownRequestPending)
{
/*
* From here on, elog(ERROR) should end with exit(1), not send control
* back to the sigsetjmp block above
*/
ExitOnAnyError = true;
/*
* Close down the database.
*
* Since ShutdownXLOG() creates restartpoint or checkpoint, and
* updates the statistics, increment the checkpoint request and flush
* out pending statistic.
*/
PendingCheckpointerStats.num_requested++;
ShutdownXLOG(0, 0);
pgstat_report_checkpointer();
pgstat_report_wal(true);
/* Normal exit from the checkpointer is here */
proc_exit(0); /* done */
}
/* Perform logging of memory contexts of this process */
if (LogMemoryContextPending)
ProcessLogMemoryContextInterrupt();
}
2011-11-01 19:07:29 +01:00
/*
* CheckArchiveTimeout -- check for archive_timeout and switch xlog files
*
* This will switch to a new WAL file and force an archive file write if
* meaningful activity is recorded in the current WAL file. This includes most
* writes, including just a single checkpoint record, but excludes WAL records
* that were inserted with the XLOG_MARK_UNIMPORTANT flag being set (like
* snapshots of running transactions). Such records, depending on
* configuration, occur on regular intervals and don't contain important
* information. This avoids generating archives with a few unimportant
* records.
2011-11-01 19:07:29 +01:00
*/
static void
CheckArchiveTimeout(void)
{
pg_time_t now;
pg_time_t last_time;
XLogRecPtr last_switch_lsn;
2011-11-01 19:07:29 +01:00
if (XLogArchiveTimeout <= 0 || RecoveryInProgress())
return;
now = (pg_time_t) time(NULL);
/* First we do a quick check using possibly-stale local state. */
if ((int) (now - last_xlog_switch_time) < XLogArchiveTimeout)
return;
/*
* Update local state ... note that last_xlog_switch_time is the last time
* a switch was performed *or requested*.
*/
last_time = GetLastSegSwitchData(&last_switch_lsn);
2011-11-01 19:07:29 +01:00
last_xlog_switch_time = Max(last_xlog_switch_time, last_time);
/* Now we can do the real checks */
2011-11-01 19:07:29 +01:00
if ((int) (now - last_xlog_switch_time) >= XLogArchiveTimeout)
{
/*
* Switch segment only when "important" WAL has been logged since the
* last segment switch (last_switch_lsn points to end of segment
* switch occurred in).
2011-11-01 19:07:29 +01:00
*/
if (GetLastImportantRecPtr() > last_switch_lsn)
{
XLogRecPtr switchpoint;
/* mark switch as unimportant, avoids triggering checkpoints */
switchpoint = RequestXLogSwitch(true);
/*
* If the returned pointer points exactly to a segment boundary,
* assume nothing happened.
*/
if (XLogSegmentOffset(switchpoint, wal_segment_size) != 0)
elog(DEBUG1, "write-ahead log switch forced (archive_timeout=%d)",
XLogArchiveTimeout);
}
2011-11-01 19:07:29 +01:00
/*
* Update state in any case, so we don't retry constantly when the
* system is idle.
*/
last_xlog_switch_time = now;
}
}
/*
* Returns true if an immediate checkpoint request is pending. (Note that
* this does not check the *current* checkpoint's IMMEDIATE flag, but whether
* there is one pending behind it.)
*/
static bool
ImmediateCheckpointRequested(void)
{
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
volatile CheckpointerShmemStruct *cps = CheckpointerShmem;
2011-11-01 19:07:29 +01:00
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
/*
* We don't need to acquire the ckpt_lck in this case because we're only
* looking at a single flag bit.
*/
if (cps->ckpt_flags & CHECKPOINT_IMMEDIATE)
return true;
2011-11-01 19:07:29 +01:00
return false;
}
/*
* CheckpointWriteDelay -- control rate of checkpoint
*
* This function is called after each page write performed by BufferSync().
* It is responsible for throttling BufferSync()'s write rate to hit
* checkpoint_completion_target.
*
* The checkpoint request flags should be passed in; currently the only one
* examined is CHECKPOINT_IMMEDIATE, which disables delays between writes.
*
* 'progress' is an estimate of how much of the work has been done, as a
* fraction between 0.0 meaning none, and 1.0 meaning all done.
*/
void
CheckpointWriteDelay(int flags, double progress)
{
static int absorb_counter = WRITES_PER_ABSORB;
/* Do nothing if checkpoint is being executed by non-checkpointer process */
Fix management of pendingOpsTable in auxiliary processes. mdinit() was misusing IsBootstrapProcessingMode() to decide whether to create an fsync pending-operations table in the current process. This led to creating a table not only in the startup and checkpointer processes as intended, but also in the bgwriter process, not to mention other auxiliary processes such as walwriter and walreceiver. Creation of the table in the bgwriter is fatal, because it absorbs fsync requests that should have gone to the checkpointer; instead they just sit in bgwriter local memory and are never acted on. So writes performed by the bgwriter were not being fsync'd which could result in data loss after an OS crash. I think there is no live bug with respect to walwriter and walreceiver because those never perform any writes of shared buffers; but the potential is there for future breakage in those processes too. To fix, make AuxiliaryProcessMain() export the current process's AuxProcType as a global variable, and then make mdinit() test directly for the types of aux process that should have a pendingOpsTable. Having done that, we might as well also get rid of the random bool flags such as am_walreceiver that some of the aux processes had grown. (Note that we could not have fixed the bug by examining those variables in mdinit(), because it's called from BaseInit() which is run by AuxiliaryProcessMain() before entering any of the process-type-specific code.) Back-patch to 9.2, where the problem was introduced by the split-up of bgwriter and checkpointer processes. The bogus pendingOpsTable exists in walwriter and walreceiver processes in earlier branches, but absent any evidence that it causes actual problems there, I'll leave the older branches alone.
2012-07-18 21:28:10 +02:00
if (!AmCheckpointerProcess())
2011-11-01 19:07:29 +01:00
return;
/*
* Perform the usual duties and take a nap, unless we're behind schedule,
* in which case we just try to catch up as quickly as possible.
*/
if (!(flags & CHECKPOINT_IMMEDIATE) &&
!ShutdownRequestPending &&
2011-11-01 19:07:29 +01:00
!ImmediateCheckpointRequested() &&
IsCheckpointOnSchedule(progress))
{
if (ConfigReloadPending)
2011-11-01 19:07:29 +01:00
{
ConfigReloadPending = false;
2011-11-01 19:07:29 +01:00
ProcessConfigFile(PGC_SIGHUP);
/* update shmem copies of config variables */
UpdateSharedMemoryConfig();
2011-11-01 19:07:29 +01:00
}
AbsorbSyncRequests();
2011-11-01 19:07:29 +01:00
absorb_counter = WRITES_PER_ABSORB;
CheckArchiveTimeout();
/* Report interim statistics to the cumulative stats system */
pgstat_report_checkpointer();
2011-11-01 19:07:29 +01:00
/*
* This sleep used to be connected to bgwriter_delay, typically 200ms.
2011-11-01 19:07:29 +01:00
* That resulted in more frequent wakeups if not much work to do.
* Checkpointer and bgwriter are no longer related so take the Big
* Sleep.
*/
WaitLatch(MyLatch, WL_LATCH_SET | WL_EXIT_ON_PM_DEATH | WL_TIMEOUT,
100,
WAIT_EVENT_CHECKPOINT_WRITE_DELAY);
ResetLatch(MyLatch);
2011-11-01 19:07:29 +01:00
}
else if (--absorb_counter <= 0)
{
/*
* Absorb pending fsync requests after each WRITES_PER_ABSORB write
* operations even when we don't sleep, to prevent overflow of the
* fsync request queue.
*/
AbsorbSyncRequests();
2011-11-01 19:07:29 +01:00
absorb_counter = WRITES_PER_ABSORB;
}
/* Check for barrier events. */
if (ProcSignalBarrierPending)
ProcessProcSignalBarrier();
2011-11-01 19:07:29 +01:00
}
/*
* IsCheckpointOnSchedule -- are we on schedule to finish this checkpoint
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
* (or restartpoint) in time?
2011-11-01 19:07:29 +01:00
*
* Compares the current progress against the time/segments elapsed since last
* checkpoint, and returns true if the progress we've made this far is greater
* than the elapsed time/segments.
*/
static bool
IsCheckpointOnSchedule(double progress)
{
XLogRecPtr recptr;
struct timeval now;
double elapsed_xlogs,
elapsed_time;
Assert(ckpt_active);
/* Scale progress according to checkpoint_completion_target. */
progress *= CheckPointCompletionTarget;
/*
* Check against the cached value first. Only do the more expensive
* calculations once we reach the target previously calculated. Since
* neither time or WAL insert pointer moves backwards, a freshly
* calculated value can only be greater than or equal to the cached value.
*/
if (progress < ckpt_cached_elapsed)
return false;
/*
* Check progress against WAL segments written and CheckPointSegments.
2011-11-01 19:07:29 +01:00
*
* We compare the current WAL insert location against the location
* computed before calling CreateCheckPoint. The code in XLogInsert that
* actually triggers a checkpoint when CheckPointSegments is exceeded
* compares against RedoRecPtr, so this is not completely accurate.
2011-11-01 19:07:29 +01:00
* However, it's good enough for our purposes, we're only calculating an
* estimate anyway.
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
*
* During recovery, we compare last replayed WAL record's location with
* the location computed before calling CreateRestartPoint. That maintains
* the same pacing as we have during checkpoints in normal operation, but
* we might exceed max_wal_size by a fair amount. That's because there can
* be a large gap between a checkpoint's redo-pointer and the checkpoint
* record itself, and we only start the restartpoint after we've seen the
* checkpoint record. (The gap is typically up to CheckPointSegments *
* checkpoint_completion_target where checkpoint_completion_target is the
* value that was in effect when the WAL was generated).
2011-11-01 19:07:29 +01:00
*/
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
if (RecoveryInProgress())
recptr = GetXLogReplayRecPtr(NULL);
else
2011-11-01 19:07:29 +01:00
recptr = GetInsertRecPtr();
elapsed_xlogs = (((double) (recptr - ckpt_start_recptr)) /
wal_segment_size) / CheckPointSegments;
2011-11-01 19:07:29 +01:00
Also trigger restartpoints based on max_wal_size on standby. When archive recovery and restartpoints were initially introduced, checkpoint_segments was ignored on the grounds that the files restored from archive don't consume any space in the recovery server. That was changed in later releases, but even then it was arguably a feature rather than a bug, as performing restartpoints as often as checkpoints during normal operation might be excessive, but you might nevertheless not want to waste a lot of space for pre-allocated WAL by setting checkpoint_segments to a high value. But now that we have separate min_wal_size and max_wal_size settings, you can bound WAL usage with max_wal_size, and still avoid consuming excessive space usage by setting min_wal_size to a lower value, so that argument is moot. There are still some issues with actually limiting the space usage to max_wal_size: restartpoints in recovery can only start after seeing the checkpoint record, while a checkpoint starts flushing buffers as soon as the redo-pointer is set. Restartpoint is paced to happen at the same leisurily speed, determined by checkpoint_completion_target, as checkpoints, but because they are started later, max_wal_size can be exceeded by upto one checkpoint cycle's worth of WAL, depending on checkpoint_completion_target. But that seems better than not trying at all, and max_wal_size is a soft limit anyway. The documentation already claimed that max_wal_size is obeyed in recovery, so this just fixes the behaviour to match the docs. However, add some weasel-words there to mention that max_wal_size may well be exceeded by some amount in recovery.
2015-06-28 23:09:10 +02:00
if (progress < elapsed_xlogs)
{
ckpt_cached_elapsed = elapsed_xlogs;
return false;
2011-11-01 19:07:29 +01:00
}
/*
* Check progress against time elapsed and checkpoint_timeout.
*/
gettimeofday(&now, NULL);
elapsed_time = ((double) ((pg_time_t) now.tv_sec - ckpt_start_time) +
now.tv_usec / 1000000.0) / CheckPointTimeout;
if (progress < elapsed_time)
{
ckpt_cached_elapsed = elapsed_time;
return false;
}
/* It looks like we're on schedule. */
return true;
}
/* --------------------------------
* signal handler routines
* --------------------------------
*/
/* SIGINT: set flag to run a normal checkpoint right away */
static void
ReqCheckpointHandler(SIGNAL_ARGS)
{
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
/*
2020-06-07 15:06:51 +02:00
* The signaling process should have set ckpt_flags nonzero, so all we
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
* need do is ensure that our main loop gets kicked out of any wait.
*/
SetLatch(MyLatch);
}
2011-11-01 19:07:29 +01:00
/* --------------------------------
* communication with backends
* --------------------------------
*/
/*
* CheckpointerShmemSize
* Compute space needed for checkpointer-related shared memory
2011-11-01 19:07:29 +01:00
*/
Size
CheckpointerShmemSize(void)
2011-11-01 19:07:29 +01:00
{
Size size;
/*
* Currently, the size of the requests[] array is arbitrarily set equal to
* NBuffers. This may prove too large or small ...
*/
size = offsetof(CheckpointerShmemStruct, requests);
size = add_size(size, mul_size(NBuffers, sizeof(CheckpointerRequest)));
2011-11-01 19:07:29 +01:00
return size;
}
/*
* CheckpointerShmemInit
* Allocate and initialize checkpointer-related shared memory
2011-11-01 19:07:29 +01:00
*/
void
CheckpointerShmemInit(void)
2011-11-01 19:07:29 +01:00
{
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
Size size = CheckpointerShmemSize();
2011-11-01 19:07:29 +01:00
bool found;
CheckpointerShmem = (CheckpointerShmemStruct *)
ShmemInitStruct("Checkpointer Data",
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
size,
2011-11-01 19:07:29 +01:00
&found);
if (!found)
{
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
/*
* First time through, so initialize. Note that we zero the whole
* requests array; this is so that CompactCheckpointerRequestQueue can
* assume that any pad bytes in the request structs are zeroes.
*/
MemSet(CheckpointerShmem, 0, size);
SpinLockInit(&CheckpointerShmem->ckpt_lck);
CheckpointerShmem->max_requests = NBuffers;
ConditionVariableInit(&CheckpointerShmem->start_cv);
ConditionVariableInit(&CheckpointerShmem->done_cv);
2011-11-01 19:07:29 +01:00
}
}
/*
* RequestCheckpoint
* Called in backend processes to request a checkpoint
*
* flags is a bitwise OR of the following:
* CHECKPOINT_IS_SHUTDOWN: checkpoint is for database shutdown.
* CHECKPOINT_END_OF_RECOVERY: checkpoint is for end of WAL recovery.
* CHECKPOINT_IMMEDIATE: finish the checkpoint ASAP,
* ignoring checkpoint_completion_target parameter.
* CHECKPOINT_FORCE: force a checkpoint even if no XLOG activity has occurred
2011-11-01 19:07:29 +01:00
* since the last one (implied by CHECKPOINT_IS_SHUTDOWN or
* CHECKPOINT_END_OF_RECOVERY).
* CHECKPOINT_WAIT: wait for completion before returning (otherwise,
* just signal checkpointer to do it, and return).
2011-11-01 19:07:29 +01:00
* CHECKPOINT_CAUSE_XLOG: checkpoint is requested due to xlog filling.
* (This affects logging, and in particular enables CheckPointWarning.)
*/
void
RequestCheckpoint(int flags)
{
int ntries;
int old_failed,
old_started;
/*
* If in a standalone backend, just do it ourselves.
*/
if (!IsPostmasterEnvironment)
{
/*
* There's no point in doing slow checkpoints in a standalone backend,
* because there's no other backends the checkpoint could disrupt.
*/
CreateCheckPoint(flags | CHECKPOINT_IMMEDIATE);
Give SMgrRelation pointers a well-defined lifetime. After calling smgropen(), it was not clear how long you could continue to use the result, because various code paths including cache invalidation could call smgrclose(), which freed the memory. Guarantee that the object won't be destroyed until the end of the current transaction, or in recovery, the commit/abort record that destroys the underlying storage. smgrclose() is now just an alias for smgrrelease(). It closes files and forgets all state except the rlocator, but keeps the SMgrRelation object valid. A new smgrdestroy() function is used by rare places that know there should be no other references to the SMgrRelation. The short version: * smgrclose() is now just an alias for smgrrelease(). It releases resources, but doesn't destroy until EOX * smgrdestroy() now frees memory, and should rarely be used. Existing code should be unaffected, but it is now possible for code that has an SMgrRelation object to use it repeatedly during a transaction as long as the storage hasn't been physically dropped. Such code would normally hold a lock on the relation. This also replaces the "ownership" mechanism of SMgrRelations with a pin counter. An SMgrRelation can now be "pinned", which prevents it from being destroyed at end of transaction. There can be multiple pins on the same SMgrRelation. In practice, the pin mechanism is only used by the relcache, so there cannot be more than one pin on the same SMgrRelation. Except with swap_relation_files XXX Author: Thomas Munro, Heikki Linnakangas Reviewed-by: Robert Haas <robertmhaas@gmail.com> Discussion: https://www.postgresql.org/message-id/CA%2BhUKGJ8NTvqLHz6dqbQnt2c8XCki4r2QvXjBQcXpVwxTY_pvA@mail.gmail.com
2024-01-31 11:31:02 +01:00
/* Free all smgr objects, as CheckpointerMain() normally would. */
smgrdestroyall();
2011-11-01 19:07:29 +01:00
return;
}
/*
* Atomically set the request flags, and take a snapshot of the counters.
* When we see ckpt_started > old_started, we know the flags we set here
* have been seen by checkpointer.
2011-11-01 19:07:29 +01:00
*
* Note that we OR the flags with any existing flags, to avoid overriding
* a "stronger" request by another backend. The flag senses must be
* chosen to make this work!
*/
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
old_failed = CheckpointerShmem->ckpt_failed;
old_started = CheckpointerShmem->ckpt_started;
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
CheckpointerShmem->ckpt_flags |= (flags | CHECKPOINT_REQUESTED);
2011-11-01 19:07:29 +01:00
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
/*
* Send signal to request checkpoint. It's possible that the checkpointer
2011-11-01 19:07:29 +01:00
* hasn't started yet, or is in process of restarting, so we will retry a
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
* few times if needed. (Actually, more than a few times, since on slow
* or overloaded buildfarm machines, it's been observed that the
* checkpointer can take several seconds to start.) However, if not told
* to wait for the checkpoint to occur, we consider failure to send the
* signal to be nonfatal and merely LOG it. The checkpointer should see
* the request when it does start, with or without getting a signal.
2011-11-01 19:07:29 +01:00
*/
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
#define MAX_SIGNAL_TRIES 600 /* max wait 60.0 sec */
2011-11-01 19:07:29 +01:00
for (ntries = 0;; ntries++)
{
if (CheckpointerShmem->checkpointer_pid == 0)
2011-11-01 19:07:29 +01:00
{
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
if (ntries >= MAX_SIGNAL_TRIES || !(flags & CHECKPOINT_WAIT))
2011-11-01 19:07:29 +01:00
{
elog((flags & CHECKPOINT_WAIT) ? ERROR : LOG,
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
"could not signal for checkpoint: checkpointer is not running");
2011-11-01 19:07:29 +01:00
break;
}
}
else if (kill(CheckpointerShmem->checkpointer_pid, SIGINT) != 0)
2011-11-01 19:07:29 +01:00
{
Make checkpoint requests more robust. Commit 6f6a6d8b1 introduced a delay of up to 2 seconds if we're trying to request a checkpoint but the checkpointer hasn't started yet (or, much less likely, our kill() call fails). However buildfarm experience shows that that's not quite enough for slow or heavily-loaded machines. There's no good reason to assume that the checkpointer won't start eventually, so we may as well make the timeout much longer, say 60 sec. However, if the caller didn't say CHECKPOINT_WAIT, it seems like a bad idea to be waiting at all, much less for as long as 60 sec. We can remove the need for that, and make this whole thing more robust, by adjusting the code so that the existence of a pending checkpoint request is clear from the contents of shared memory, and making sure that the checkpointer process will notice it at startup even if it did not get a signal. In this way there's no need for a non-CHECKPOINT_WAIT call to wait at all; if it can't send the signal, it can nonetheless assume that the checkpointer will eventually service the request. A potential downside of this change is that "kill -INT" on the checkpointer process is no longer enough to trigger a checkpoint, should anyone be relying on something so hacky. But there's no obvious reason to do it like that rather than issuing a plain old CHECKPOINT command, so we'll assume that nobody is. There doesn't seem to be a way to preserve this undocumented quasi-feature without introducing race conditions. Since a principal reason for messing with this is to prevent intermittent buildfarm failures, back-patch to all supported branches. Discussion: https://postgr.es/m/27830.1552752475@sss.pgh.pa.us
2019-03-19 17:49:27 +01:00
if (ntries >= MAX_SIGNAL_TRIES || !(flags & CHECKPOINT_WAIT))
2011-11-01 19:07:29 +01:00
{
elog((flags & CHECKPOINT_WAIT) ? ERROR : LOG,
"could not signal for checkpoint: %m");
break;
}
}
else
break; /* signal sent successfully */
CHECK_FOR_INTERRUPTS();
pg_usleep(100000L); /* wait 0.1 sec, then retry */
}
/*
* If requested, wait for completion. We detect completion according to
* the algorithm given above.
*/
if (flags & CHECKPOINT_WAIT)
{
int new_started,
new_failed;
/* Wait for a new checkpoint to start. */
ConditionVariablePrepareToSleep(&CheckpointerShmem->start_cv);
2011-11-01 19:07:29 +01:00
for (;;)
{
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
new_started = CheckpointerShmem->ckpt_started;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
if (new_started != old_started)
break;
ConditionVariableSleep(&CheckpointerShmem->start_cv,
WAIT_EVENT_CHECKPOINT_START);
2011-11-01 19:07:29 +01:00
}
ConditionVariableCancelSleep();
2011-11-01 19:07:29 +01:00
/*
* We are waiting for ckpt_done >= new_started, in a modulo sense.
*/
ConditionVariablePrepareToSleep(&CheckpointerShmem->done_cv);
2011-11-01 19:07:29 +01:00
for (;;)
{
int new_done;
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
new_done = CheckpointerShmem->ckpt_done;
new_failed = CheckpointerShmem->ckpt_failed;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
2011-11-01 19:07:29 +01:00
if (new_done - new_started >= 0)
break;
ConditionVariableSleep(&CheckpointerShmem->done_cv,
WAIT_EVENT_CHECKPOINT_DONE);
2011-11-01 19:07:29 +01:00
}
ConditionVariableCancelSleep();
2011-11-01 19:07:29 +01:00
if (new_failed != old_failed)
ereport(ERROR,
(errmsg("checkpoint request failed"),
errhint("Consult recent messages in the server log for details.")));
}
}
/*
* ForwardSyncRequest
* Forward a file-fsync request from a backend to the checkpointer
2011-11-01 19:07:29 +01:00
*
* Whenever a backend is compelled to write directly to a relation
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
* (which should be seldom, if the background writer is getting its job done),
2011-11-01 19:07:29 +01:00
* the backend calls this routine to pass over knowledge that the relation
* is dirty and must be fsync'd before next checkpoint. We also use this
* opportunity to count such writes for statistical purposes.
*
* To avoid holding the lock for longer than necessary, we normally write
* to the requests[] queue without checking for duplicates. The checkpointer
2011-11-01 19:07:29 +01:00
* will have to eliminate dups internally anyway. However, if we discover
* that the queue is full, we make a pass over the entire queue to compact
* it. This is somewhat expensive, but the alternative is for the backend
* to perform its own fsync, which is far more expensive in practice. It
* is theoretically possible a backend fsync might still be necessary, if
* the queue is full and contains no duplicate entries. In that case, we
* let the backend know by returning false.
*/
bool
ForwardSyncRequest(const FileTag *ftag, SyncRequestType type)
2011-11-01 19:07:29 +01:00
{
CheckpointerRequest *request;
bool too_full;
2011-11-01 19:07:29 +01:00
if (!IsUnderPostmaster)
return false; /* probably shouldn't even get here */
Fix management of pendingOpsTable in auxiliary processes. mdinit() was misusing IsBootstrapProcessingMode() to decide whether to create an fsync pending-operations table in the current process. This led to creating a table not only in the startup and checkpointer processes as intended, but also in the bgwriter process, not to mention other auxiliary processes such as walwriter and walreceiver. Creation of the table in the bgwriter is fatal, because it absorbs fsync requests that should have gone to the checkpointer; instead they just sit in bgwriter local memory and are never acted on. So writes performed by the bgwriter were not being fsync'd which could result in data loss after an OS crash. I think there is no live bug with respect to walwriter and walreceiver because those never perform any writes of shared buffers; but the potential is there for future breakage in those processes too. To fix, make AuxiliaryProcessMain() export the current process's AuxProcType as a global variable, and then make mdinit() test directly for the types of aux process that should have a pendingOpsTable. Having done that, we might as well also get rid of the random bool flags such as am_walreceiver that some of the aux processes had grown. (Note that we could not have fixed the bug by examining those variables in mdinit(), because it's called from BaseInit() which is run by AuxiliaryProcessMain() before entering any of the process-type-specific code.) Back-patch to 9.2, where the problem was introduced by the split-up of bgwriter and checkpointer processes. The bogus pendingOpsTable exists in walwriter and walreceiver processes in earlier branches, but absent any evidence that it causes actual problems there, I'll leave the older branches alone.
2012-07-18 21:28:10 +02:00
if (AmCheckpointerProcess())
elog(ERROR, "ForwardSyncRequest must not be called in checkpointer");
2011-11-01 19:07:29 +01:00
LWLockAcquire(CheckpointerCommLock, LW_EXCLUSIVE);
2011-11-01 19:07:29 +01:00
/*
* If the checkpointer isn't running or the request queue is full, the
2011-11-01 19:07:29 +01:00
* backend will have to perform its own fsync request. But before forcing
* that to happen, we can try to compact the request queue.
2011-11-01 19:07:29 +01:00
*/
if (CheckpointerShmem->checkpointer_pid == 0 ||
(CheckpointerShmem->num_requests >= CheckpointerShmem->max_requests &&
!CompactCheckpointerRequestQueue()))
2011-11-01 19:07:29 +01:00
{
LWLockRelease(CheckpointerCommLock);
2011-11-01 19:07:29 +01:00
return false;
}
/* OK, insert request */
request = &CheckpointerShmem->requests[CheckpointerShmem->num_requests++];
request->ftag = *ftag;
request->type = type;
/* If queue is more than half full, nudge the checkpointer to empty it */
too_full = (CheckpointerShmem->num_requests >=
CheckpointerShmem->max_requests / 2);
LWLockRelease(CheckpointerCommLock);
/* ... but not till after we release the lock */
if (too_full && ProcGlobal->checkpointerLatch)
SetLatch(ProcGlobal->checkpointerLatch);
2011-11-01 19:07:29 +01:00
return true;
}
/*
* CompactCheckpointerRequestQueue
* Remove duplicates from the request queue to avoid backend fsyncs.
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
* Returns "true" if any entries were removed.
2011-11-01 19:07:29 +01:00
*
* Although a full fsync request queue is not common, it can lead to severe
* performance problems when it does happen. So far, this situation has
* only been observed to occur when the system is under heavy write load,
* and especially during the "sync" phase of a checkpoint. Without this
* logic, each backend begins doing an fsync for every block written, which
* gets very expensive and can slow down the whole system.
*
* Trying to do this every time the queue is full could lose if there
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
* aren't any removable entries. But that should be vanishingly rare in
2011-11-01 19:07:29 +01:00
* practice: there's one queue entry per shared buffer.
*/
static bool
CompactCheckpointerRequestQueue(void)
2011-11-01 19:07:29 +01:00
{
struct CheckpointerSlotMapping
2011-11-01 19:07:29 +01:00
{
CheckpointerRequest request;
2011-11-01 19:07:29 +01:00
int slot;
};
int n,
preserve_count;
int num_skipped = 0;
HASHCTL ctl;
HTAB *htab;
bool *skip_slot;
/* must hold CheckpointerCommLock in exclusive mode */
Assert(LWLockHeldByMe(CheckpointerCommLock));
2011-11-01 19:07:29 +01:00
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
/* Initialize skip_slot array */
skip_slot = palloc0(sizeof(bool) * CheckpointerShmem->num_requests);
2011-11-01 19:07:29 +01:00
/* Initialize temporary hash table */
ctl.keysize = sizeof(CheckpointerRequest);
ctl.entrysize = sizeof(struct CheckpointerSlotMapping);
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
ctl.hcxt = CurrentMemoryContext;
htab = hash_create("CompactCheckpointerRequestQueue",
CheckpointerShmem->num_requests,
2011-11-01 19:07:29 +01:00
&ctl,
Improve hash_create's API for selecting simple-binary-key hash functions. Previously, if you wanted anything besides C-string hash keys, you had to specify a custom hashing function to hash_create(). Nearly all such callers were specifying tag_hash or oid_hash; which is tedious, and rather error-prone, since a caller could easily miss the opportunity to optimize by using hash_uint32 when appropriate. Replace this with a design whereby callers using simple binary-data keys just specify HASH_BLOBS and don't need to mess with specific support functions. hash_create() itself will take care of optimizing when the key size is four bytes. This nets out saving a few hundred bytes of code space, and offers a measurable performance improvement in tidbitmap.c (which was not exploiting the opportunity to use hash_uint32 for its 4-byte keys). There might be some wins elsewhere too, I didn't analyze closely. In future we could look into offering a similar optimized hashing function for 8-byte keys. Under this design that could be done in a centralized and machine-independent fashion, whereas getting it right for keys of platform-dependent sizes would've been notationally painful before. For the moment, the old way still works fine, so as not to break source code compatibility for loadable modules. Eventually we might want to remove tag_hash and friends from the exported API altogether, since there's no real need for them to be explicitly referenced from outside dynahash.c. Teodor Sigaev and Tom Lane
2014-12-18 19:36:29 +01:00
HASH_ELEM | HASH_BLOBS | HASH_CONTEXT);
2011-11-01 19:07:29 +01:00
/*
* The basic idea here is that a request can be skipped if it's followed
* by a later, identical request. It might seem more sensible to work
* backwards from the end of the queue and check whether a request is
* *preceded* by an earlier, identical request, in the hopes of doing less
* copying. But that might change the semantics, if there's an
* intervening SYNC_FORGET_REQUEST or SYNC_FILTER_REQUEST, so we do it
* this way. It would be possible to be even smarter if we made the code
* below understand the specific semantics of such requests (it could blow
* away preceding entries that would end up being canceled anyhow), but
* it's not clear that the extra complexity would buy us anything.
2011-11-01 19:07:29 +01:00
*/
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
for (n = 0; n < CheckpointerShmem->num_requests; n++)
2011-11-01 19:07:29 +01:00
{
CheckpointerRequest *request;
struct CheckpointerSlotMapping *slotmap;
2011-11-01 19:07:29 +01:00
bool found;
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
/*
* We use the request struct directly as a hashtable key. This
* assumes that any padding bytes in the structs are consistently the
* same, which should be okay because we zeroed them in
Change internal RelFileNode references to RelFileNumber or RelFileLocator. We have been using the term RelFileNode to refer to either (1) the integer that is used to name the sequence of files for a certain relation within the directory set aside for that tablespace/database combination; or (2) that value plus the OIDs of the tablespace and database; or occasionally (3) the whole series of files created for a relation based on those values. Using the same name for more than one thing is confusing. Replace RelFileNode with RelFileNumber when we're talking about just the single number, i.e. (1) from above, and with RelFileLocator when we're talking about all the things that are needed to locate a relation's files on disk, i.e. (2) from above. In the places where we refer to (3) as a relfilenode, instead refer to "relation storage". Since there is a ton of SQL code in the world that knows about pg_class.relfilenode, don't change the name of that column, or of other SQL-facing things that derive their name from it. On the other hand, do adjust closely-related internal terminology. For example, the structure member names dbNode and spcNode appear to be derived from the fact that the structure itself was called RelFileNode, so change those to dbOid and spcOid. Likewise, various variables with names like rnode and relnode get renamed appropriately, according to how they're being used in context. Hopefully, this is clearer than before. It is also preparation for future patches that intend to widen the relfilenumber fields from its current width of 32 bits. Variables that store a relfilenumber are now declared as type RelFileNumber rather than type Oid; right now, these are the same, but that can now more easily be changed. Dilip Kumar, per an idea from me. Reviewed also by Andres Freund. I fixed some whitespace issues, changed a couple of words in a comment, and made one other minor correction. Discussion: http://postgr.es/m/CA+TgmoamOtXbVAQf9hWFzonUo6bhhjS6toZQd7HZ-pmojtAmag@mail.gmail.com Discussion: http://postgr.es/m/CA+Tgmobp7+7kmi4gkq7Y+4AM9fTvL+O1oQ4-5gFTT+6Ng-dQ=g@mail.gmail.com Discussion: http://postgr.es/m/CAFiTN-vTe79M8uDH1yprOU64MNFE+R3ODRuA+JWf27JbhY4hJw@mail.gmail.com
2022-07-06 17:39:09 +02:00
* CheckpointerShmemInit. Note also that RelFileLocator had better
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
* contain no pad bytes.
*/
request = &CheckpointerShmem->requests[n];
2011-11-01 19:07:29 +01:00
slotmap = hash_search(htab, request, HASH_ENTER, &found);
if (found)
{
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
/* Duplicate, so mark the previous occurrence as skippable */
2011-11-01 19:07:29 +01:00
skip_slot[slotmap->slot] = true;
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
num_skipped++;
2011-11-01 19:07:29 +01:00
}
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
/* Remember slot containing latest occurrence of this request value */
2011-11-01 19:07:29 +01:00
slotmap->slot = n;
}
/* Done with the hash table. */
hash_destroy(htab);
/* If no duplicates, we're out of luck. */
if (!num_skipped)
{
pfree(skip_slot);
return false;
}
/* We found some duplicates; remove them. */
Improve coding around the fsync request queue. In all branches back to 8.3, this patch fixes a questionable assumption in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue that there are no uninitialized pad bytes in the request queue structs. This would only cause trouble if (a) there were such pad bytes, which could happen in 8.4 and up if the compiler makes enum ForkNumber narrower than 32 bits, but otherwise would require not-currently-planned changes in the widths of other typedefs; and (b) the kernel has not uniformly initialized the contents of shared memory to zeroes. Still, it seems a tad risky, and we can easily remove any risk by pre-zeroing the request array for ourselves. In addition to that, we need to establish a coding rule that struct RelFileNode can't contain any padding bytes, since such structs are copied into the request array verbatim. (There are other places that are assuming this anyway, it turns out.) In 9.1 and up, the risk was a bit larger because we were also effectively assuming that struct RelFileNodeBackend contained no pad bytes, and with fields of different types in there, that would be much easier to break. However, there is no good reason to ever transmit fsync or delete requests for temp files to the bgwriter/checkpointer, so we can revert the request structs to plain RelFileNode, getting rid of the padding risk and saving some marginal number of bytes and cycles in fsync queue manipulation while we are at it. The savings might be more than marginal during deletion of a temp relation, because the old code transmitted an entirely useless but nonetheless expensive-to-process ForgetRelationFsync request to the background process, and also had the background process perform the file deletion even though that can safely be done immediately. In addition, make some cleanup of nearby comments and small improvements to the code in CompactCheckpointerRequestQueue/CompactBgwriterRequestQueue.
2012-07-17 22:55:39 +02:00
preserve_count = 0;
for (n = 0; n < CheckpointerShmem->num_requests; n++)
2011-11-01 19:07:29 +01:00
{
if (skip_slot[n])
continue;
CheckpointerShmem->requests[preserve_count++] = CheckpointerShmem->requests[n];
2011-11-01 19:07:29 +01:00
}
ereport(DEBUG1,
(errmsg_internal("compacted fsync request queue from %d entries to %d entries",
CheckpointerShmem->num_requests, preserve_count)));
CheckpointerShmem->num_requests = preserve_count;
2011-11-01 19:07:29 +01:00
/* Cleanup. */
pfree(skip_slot);
return true;
}
/*
* AbsorbSyncRequests
* Retrieve queued sync requests and pass them to sync mechanism.
2011-11-01 19:07:29 +01:00
*
* This is exported because it must be called during CreateCheckPoint;
* we have to be sure we have accepted all pending requests just before
* we start fsync'ing. Since CreateCheckPoint sometimes runs in
* non-checkpointer processes, do nothing if not checkpointer.
*/
void
AbsorbSyncRequests(void)
2011-11-01 19:07:29 +01:00
{
CheckpointerRequest *requests = NULL;
CheckpointerRequest *request;
2011-11-01 19:07:29 +01:00
int n;
Fix management of pendingOpsTable in auxiliary processes. mdinit() was misusing IsBootstrapProcessingMode() to decide whether to create an fsync pending-operations table in the current process. This led to creating a table not only in the startup and checkpointer processes as intended, but also in the bgwriter process, not to mention other auxiliary processes such as walwriter and walreceiver. Creation of the table in the bgwriter is fatal, because it absorbs fsync requests that should have gone to the checkpointer; instead they just sit in bgwriter local memory and are never acted on. So writes performed by the bgwriter were not being fsync'd which could result in data loss after an OS crash. I think there is no live bug with respect to walwriter and walreceiver because those never perform any writes of shared buffers; but the potential is there for future breakage in those processes too. To fix, make AuxiliaryProcessMain() export the current process's AuxProcType as a global variable, and then make mdinit() test directly for the types of aux process that should have a pendingOpsTable. Having done that, we might as well also get rid of the random bool flags such as am_walreceiver that some of the aux processes had grown. (Note that we could not have fixed the bug by examining those variables in mdinit(), because it's called from BaseInit() which is run by AuxiliaryProcessMain() before entering any of the process-type-specific code.) Back-patch to 9.2, where the problem was introduced by the split-up of bgwriter and checkpointer processes. The bogus pendingOpsTable exists in walwriter and walreceiver processes in earlier branches, but absent any evidence that it causes actual problems there, I'll leave the older branches alone.
2012-07-18 21:28:10 +02:00
if (!AmCheckpointerProcess())
2011-11-01 19:07:29 +01:00
return;
LWLockAcquire(CheckpointerCommLock, LW_EXCLUSIVE);
2011-11-01 19:07:29 +01:00
/*
* We try to avoid holding the lock for a long time by copying the request
* array, and processing the requests after releasing the lock.
*
* Once we have cleared the requests from shared memory, we have to PANIC
* if we then fail to absorb them (eg, because our hashtable runs out of
* memory). This is because the system cannot run safely if we are unable
* to fsync what we have been told to fsync. Fortunately, the hashtable
* is so small that the problem is quite unlikely to arise in practice.
*/
n = CheckpointerShmem->num_requests;
2011-11-01 19:07:29 +01:00
if (n > 0)
{
requests = (CheckpointerRequest *) palloc(n * sizeof(CheckpointerRequest));
memcpy(requests, CheckpointerShmem->requests, n * sizeof(CheckpointerRequest));
2011-11-01 19:07:29 +01:00
}
START_CRIT_SECTION();
CheckpointerShmem->num_requests = 0;
2011-11-01 19:07:29 +01:00
LWLockRelease(CheckpointerCommLock);
2011-11-01 19:07:29 +01:00
for (request = requests; n > 0; request++, n--)
RememberSyncRequest(&request->ftag, request->type);
2011-11-01 19:07:29 +01:00
END_CRIT_SECTION();
2011-11-01 19:07:29 +01:00
if (requests)
pfree(requests);
}
/*
* Update any shared memory configurations based on config parameters
*/
static void
UpdateSharedMemoryConfig(void)
{
/* update global shmem state for sync rep */
SyncRepUpdateSyncStandbysDefined();
/*
* If full_page_writes has been changed by SIGHUP, we update it in shared
* memory and write an XLOG_FPW_CHANGE record.
*/
UpdateFullPageWrites();
elog(DEBUG2, "checkpointer updated shared memory configuration values");
}
/*
* FirstCallSinceLastCheckpoint allows a process to take an action once
* per checkpoint cycle by asynchronously checking for checkpoint completion.
*/
bool
FirstCallSinceLastCheckpoint(void)
{
static int ckpt_done = 0;
int new_done;
bool FirstCall = false;
SpinLockAcquire(&CheckpointerShmem->ckpt_lck);
new_done = CheckpointerShmem->ckpt_done;
SpinLockRelease(&CheckpointerShmem->ckpt_lck);
if (new_done != ckpt_done)
FirstCall = true;
ckpt_done = new_done;
return FirstCall;
}