Implement "distributed" checkpoints in which the checkpoint I/O is spread

over a fairly long period of time, rather than being spat out in a burst.
This happens only for background checkpoints carried out by the bgwriter;
other cases, such as a shutdown checkpoint, are still done at full speed.

Remove the "all buffers" scan in the bgwriter, and associated stats
infrastructure, since this seems no longer very useful when the checkpoint
itself is properly throttled.

Original patch by Itagaki Takahiro, reworked by Heikki Linnakangas,
and some minor API editorialization by me.
This commit is contained in:
Tom Lane 2007-06-28 00:02:40 +00:00
parent b09c248bdd
commit 867e2c91a0
22 changed files with 760 additions and 463 deletions

View File

@ -1,4 +1,4 @@
<!-- $PostgreSQL: pgsql/doc/src/sgml/config.sgml,v 1.128 2007/06/22 16:15:23 tgl Exp $ -->
<!-- $PostgreSQL: pgsql/doc/src/sgml/config.sgml,v 1.129 2007/06/28 00:02:37 tgl Exp $ -->
<chapter Id="runtime-config">
<title>Server Configuration</title>
@ -1168,21 +1168,17 @@ SET ENABLE_SEQSCAN TO OFF;
<para>
Beginning in <productname>PostgreSQL</> 8.0, there is a separate server
process called the <firstterm>background writer</>, whose sole function
process called the <firstterm>background writer</>, whose function
is to issue writes of <quote>dirty</> shared buffers. The intent is
that server processes handling user queries should seldom or never have
to wait for a write to occur, because the background writer will do it.
This arrangement also reduces the performance penalty associated with
checkpoints. The background writer will continuously trickle out dirty
pages to disk, so that only a few pages will need to be forced out when
checkpoint time arrives, instead of the storm of dirty-buffer writes that
formerly occurred at each checkpoint. However there is a net overall
However there is a net overall
increase in I/O load, because where a repeatedly-dirtied page might
before have been written only once per checkpoint interval, the
background writer might write it several times in the same interval.
In most situations a continuous low load is preferable to periodic
spikes, but the parameters discussed in this subsection can be used to tune
the behavior for local needs.
spikes, but the parameters discussed in this subsection can be used to
tune the behavior for local needs.
</para>
<variablelist>
@ -1242,62 +1238,14 @@ SET ENABLE_SEQSCAN TO OFF;
</para>
</listitem>
</varlistentry>
<varlistentry id="guc-bgwriter-all-percent" xreflabel="bgwriter_all_percent">
<term><varname>bgwriter_all_percent</varname> (<type>floating point</type>)</term>
<indexterm>
<primary><varname>bgwriter_all_percent</> configuration parameter</primary>
</indexterm>
<listitem>
<para>
To reduce the amount of work that will be needed at checkpoint time,
the background writer also does a circular scan through the entire
buffer pool, writing buffers that are found to be dirty.
In each round, it examines up to
<varname>bgwriter_all_percent</> of the buffers for this purpose.
The default value is 0.333 (0.333% of the total number
of shared buffers). With the default <varname>bgwriter_delay</>
setting, this will allow the entire shared buffer pool to be scanned
about once per minute.
This parameter can only be set in the <filename>postgresql.conf</>
file or on the server command line.
</para>
</listitem>
</varlistentry>
<varlistentry id="guc-bgwriter-all-maxpages" xreflabel="bgwriter_all_maxpages">
<term><varname>bgwriter_all_maxpages</varname> (<type>integer</type>)</term>
<indexterm>
<primary><varname>bgwriter_all_maxpages</> configuration parameter</primary>
</indexterm>
<listitem>
<para>
In each round, no more than this many buffers will be written
as a result of the scan of the entire buffer pool. (If this
limit is reached, the scan stops, and resumes at the next buffer
during the next round.)
The default value is five buffers.
This parameter can only be set in the <filename>postgresql.conf</>
file or on the server command line.
</para>
</listitem>
</varlistentry>
</variablelist>
<para>
Smaller values of <varname>bgwriter_all_percent</varname> and
<varname>bgwriter_all_maxpages</varname> reduce the extra I/O load
caused by the background writer, but leave more work to be done
at checkpoint time. To reduce load spikes at checkpoints,
increase these two values.
Similarly, smaller values of <varname>bgwriter_lru_percent</varname> and
Smaller values of <varname>bgwriter_lru_percent</varname> and
<varname>bgwriter_lru_maxpages</varname> reduce the extra I/O load
caused by the background writer, but make it more likely that server
processes will have to issue writes for themselves, delaying interactive
queries.
To disable background writing entirely,
set both <varname>maxpages</varname> values and/or both
<varname>percent</varname> values to zero.
</para>
</sect2>
</sect1>
@ -1307,7 +1255,7 @@ SET ENABLE_SEQSCAN TO OFF;
<para>
See also <xref linkend="wal-configuration"> for details on WAL
tuning.
and checkpoint tuning.
</para>
<sect2 id="runtime-config-wal-settings">
@ -1565,6 +1513,22 @@ SET ENABLE_SEQSCAN TO OFF;
</listitem>
</varlistentry>
<varlistentry id="guc-checkpoint-completion-target" xreflabel="checkpoint_completion_target">
<term><varname>checkpoint_completion_target</varname> (<type>floating point</type>)</term>
<indexterm>
<primary><varname>checkpoint_completion_target</> configuration parameter</primary>
</indexterm>
<listitem>
<para>
Specifies the target length of checkpoints, as a fraction of
the checkpoint interval. The default is 0.5.
This parameter can only be set in the <filename>postgresql.conf</>
file or on the server command line.
</para>
</listitem>
</varlistentry>
<varlistentry id="guc-checkpoint-warning" xreflabel="checkpoint_warning">
<term><varname>checkpoint_warning</varname> (<type>integer</type>)</term>
<indexterm>

View File

@ -1,4 +1,4 @@
<!-- $PostgreSQL: pgsql/doc/src/sgml/monitoring.sgml,v 1.50 2007/04/27 20:08:43 neilc Exp $ -->
<!-- $PostgreSQL: pgsql/doc/src/sgml/monitoring.sgml,v 1.51 2007/06/28 00:02:37 tgl Exp $ -->
<chapter id="monitoring">
<title>Monitoring Database Activity</title>
@ -251,9 +251,9 @@ postgres: <replaceable>user</> <replaceable>database</> <replaceable>host</> <re
<entry><structname>pg_stat_bgwriter</></entry>
<entry>One row only, showing cluster-wide statistics from the
background writer: number of scheduled checkpoints, requested
checkpoints, buffers written by checkpoints, lru-scans and all-scans,
and the number of times the bgwriter aborted a round because it had
written too many buffers during lru-scans and all-scans.
checkpoints, buffers written by checkpoints and cleaning scans,
and the number of times the bgwriter stopped a cleaning scan
because it had written too many buffers.
</entry>
</row>
@ -777,43 +777,24 @@ postgres: <replaceable>user</> <replaceable>database</> <replaceable>host</> <re
</row>
<row>
<entry><literal><function>pg_stat_get_bgwriter_buf_written_lru</function>()</literal></entry>
<entry><literal><function>pg_stat_get_bgwriter_buf_written_clean</function>()</literal></entry>
<entry><type>bigint</type></entry>
<entry>
The number of buffers written by the bgwriter when performing a
LRU scan of the buffer cache
The number of buffers written by the bgwriter for routine cleaning of
dirty pages
</entry>
</row>
<row>
<entry><literal><function>pg_stat_get_bgwriter_buf_written_all</function>()</literal></entry>
<entry><literal><function>pg_stat_get_bgwriter_maxwritten_clean</function>()</literal></entry>
<entry><type>bigint</type></entry>
<entry>
The number of buffers written by the bgwriter when performing a
scan of all the buffer cache
</entry>
</row>
<row>
<entry><literal><function>pg_stat_get_bgwriter_maxwritten_lru</function>()</literal></entry>
<entry><type>bigint</type></entry>
<entry>
The number of times the bgwriter has stopped its LRU round because
The number of times the bgwriter has stopped its cleaning scan because
it has written more buffers than specified in the
<varname>bgwriter_lru_maxpages</varname> parameter
</entry>
</row>
<row>
<entry><literal><function>pg_stat_get_bgwriter_maxwritten_all</function>()</literal></entry>
<entry><type>bigint</type></entry>
<entry>
The number of times the bgwriter has stopped its all-buffer round
because it has written more buffers than specified in the
<varname>bgwriter_all_maxpages</varname> parameter
</entry>
</row>
<row>
<entry><literal><function>pg_stat_clear_snapshot</function>()</literal></entry>
<entry><type>void</type></entry>

View File

@ -1,4 +1,4 @@
<!-- $PostgreSQL: pgsql/doc/src/sgml/wal.sgml,v 1.43 2007/01/31 20:56:19 momjian Exp $ -->
<!-- $PostgreSQL: pgsql/doc/src/sgml/wal.sgml,v 1.44 2007/06/28 00:02:37 tgl Exp $ -->
<chapter id="wal">
<title>Reliability and the Write-Ahead Log</title>
@ -217,15 +217,41 @@
</para>
<para>
There will be at least one WAL segment file, and will normally
not be more than 2 * <varname>checkpoint_segments</varname> + 1
To avoid flooding the I/O system with a burst of page writes,
writing dirty buffers during a checkpoint is spread over a period of time.
That period is controlled by
<xref linkend="guc-checkpoint-completion-target">, which is
given as a fraction of the checkpoint interval.
The I/O rate is adjusted so that the checkpoint finishes when the
given fraction of <varname>checkpoint_segments</varname> WAL segments
have been consumed since checkpoint start, or the given fraction of
<varname>checkpoint_timeout</varname> seconds have elapsed,
whichever is sooner. With the default value of 0.5,
<productname>PostgreSQL</> can be expected to complete each checkpoint
in about half the time before the next checkpoint starts. On a system
that's very close to maximum I/O throughput during normal operation,
you might want to increase <varname>checkpoint_completion_target</varname>
to reduce the I/O load from checkpoints. The disadvantage of this is that
prolonging checkpoints affects recovery time, because more WAL segments
will need to be kept around for possible use in recovery. Although
<varname>checkpoint_completion_target</varname> can be set as high as 1.0,
it is best to keep it less than that (perhaps 0.9 at most) since
checkpoints include some other activities besides writing dirty buffers.
A setting of 1.0 is quite likely to result in checkpoints not being
completed on time, which would result in performance loss due to
unexpected variation in the number of WAL segments needed.
</para>
<para>
There will always be at least one WAL segment file, and will normally
not be more than (2 + <varname>checkpoint_completion_target</varname>) * <varname>checkpoint_segments</varname> + 1
files. Each segment file is normally 16 MB (though this size can be
altered when building the server). You can use this to estimate space
requirements for <acronym>WAL</acronym>.
Ordinarily, when old log segment files are no longer needed, they
are recycled (renamed to become the next segments in the numbered
sequence). If, due to a short-term peak of log output rate, there
are more than 2 * <varname>checkpoint_segments</varname> + 1
are more than 3 * <varname>checkpoint_segments</varname> + 1
segment files, the unneeded segment files will be deleted instead
of recycled until the system gets back under this limit.
</para>

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/backend/access/transam/xlog.c,v 1.272 2007/05/31 15:13:01 petere Exp $
* $PostgreSQL: pgsql/src/backend/access/transam/xlog.c,v 1.273 2007/06/28 00:02:37 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -72,16 +72,16 @@ bool XLOG_DEBUG = false;
#endif
/*
* XLOGfileslop is used in the code as the allowed "fuzz" in the number of
* preallocated XLOG segments --- we try to have at least XLOGfiles advance
* segments but no more than XLOGfileslop segments. This could
* be made a separate GUC variable, but at present I think it's sufficient
* to hardwire it as 2*CheckPointSegments+1. Under normal conditions, a
* checkpoint will free no more than 2*CheckPointSegments log segments, and
* we want to recycle all of them; the +1 allows boundary cases to happen
* without wasting a delete/create-segment cycle.
* XLOGfileslop is the maximum number of preallocated future XLOG segments.
* When we are done with an old XLOG segment file, we will recycle it as a
* future XLOG segment as long as there aren't already XLOGfileslop future
* segments; else we'll delete it. This could be made a separate GUC
* variable, but at present I think it's sufficient to hardwire it as
* 2*CheckPointSegments+1. Under normal conditions, a checkpoint will free
* no more than 2*CheckPointSegments log segments, and we want to recycle all
* of them; the +1 allows boundary cases to happen without wasting a
* delete/create-segment cycle.
*/
#define XLOGfileslop (2*CheckPointSegments + 1)
@ -398,7 +398,7 @@ static void readRecoveryCommandFile(void);
static void exitArchiveRecovery(TimeLineID endTLI,
uint32 endLogId, uint32 endLogSeg);
static bool recoveryStopsHere(XLogRecord *record, bool *includeThis);
static void CheckPointGuts(XLogRecPtr checkPointRedo);
static void CheckPointGuts(XLogRecPtr checkPointRedo, int flags);
static bool XLogCheckBuffer(XLogRecData *rdata, bool doPageWrites,
XLogRecPtr *lsn, BkpBlock *bkpb);
@ -415,7 +415,7 @@ static void XLogFileClose(void);
static bool RestoreArchivedFile(char *path, const char *xlogfname,
const char *recovername, off_t expectedSize);
static int PreallocXlogFiles(XLogRecPtr endptr);
static void MoveOfflineLogs(uint32 log, uint32 seg, XLogRecPtr endptr,
static void RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr,
int *nsegsremoved, int *nsegsrecycled);
static void CleanupBackupHistory(void);
static XLogRecord *ReadRecord(XLogRecPtr *RecPtr, int emode);
@ -1608,7 +1608,7 @@ XLogWrite(XLogwrtRqst WriteRqst, bool flexible, bool xlog_switch)
if (XLOG_DEBUG)
elog(LOG, "time for a checkpoint, signaling bgwriter");
#endif
RequestCheckpoint(false, true);
RequestCheckpoint(CHECKPOINT_WARNONTIME);
}
}
}
@ -2516,8 +2516,14 @@ RestoreArchivedFile(char *path, const char *xlogfname,
}
/*
* Preallocate log files beyond the specified log endpoint, according to
* the XLOGfile user parameter.
* Preallocate log files beyond the specified log endpoint.
*
* XXX this is currently extremely conservative, since it forces only one
* future log segment to exist, and even that only if we are 75% done with
* the current one. This is only appropriate for very low-WAL-volume systems.
* High-volume systems will be OK once they've built up a sufficient set of
* recycled log segments, but the startup transient is likely to include
* a lot of segment creations by foreground processes, which is not so good.
*/
static int
PreallocXlogFiles(XLogRecPtr endptr)
@ -2543,14 +2549,14 @@ PreallocXlogFiles(XLogRecPtr endptr)
}
/*
* Remove or move offline all log files older or equal to passed log/seg#
* Recycle or remove all log files older or equal to passed log/seg#
*
* endptr is current (or recent) end of xlog; this is used to determine
* whether we want to recycle rather than delete no-longer-wanted log files.
*/
static void
MoveOfflineLogs(uint32 log, uint32 seg, XLogRecPtr endptr,
int *nsegsremoved, int *nsegsrecycled)
RemoveOldXlogFiles(uint32 log, uint32 seg, XLogRecPtr endptr,
int *nsegsremoved, int *nsegsrecycled)
{
uint32 endlogId;
uint32 endlogSeg;
@ -5110,7 +5116,7 @@ StartupXLOG(void)
* the rule that TLI only changes in shutdown checkpoints, which
* allows some extra error checking in xlog_redo.
*/
CreateCheckPoint(true, true);
CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
/*
* Close down recovery environment
@ -5318,6 +5324,29 @@ GetRedoRecPtr(void)
return RedoRecPtr;
}
/*
* GetInsertRecPtr -- Returns the current insert position.
*
* NOTE: The value *actually* returned is the position of the last full
* xlog page. It lags behind the real insert position by at most 1 page.
* For that, we don't need to acquire WALInsertLock which can be quite
* heavily contended, and an approximation is enough for the current
* usage of this function.
*/
XLogRecPtr
GetInsertRecPtr(void)
{
/* use volatile pointer to prevent code rearrangement */
volatile XLogCtlData *xlogctl = XLogCtl;
XLogRecPtr recptr;
SpinLockAcquire(&xlogctl->info_lck);
recptr = xlogctl->LogwrtRqst.Write;
SpinLockRelease(&xlogctl->info_lck);
return recptr;
}
/*
* Get the time of the last xlog segment switch
*/
@ -5383,7 +5412,7 @@ ShutdownXLOG(int code, Datum arg)
ereport(LOG,
(errmsg("shutting down")));
CreateCheckPoint(true, true);
CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
ShutdownCLOG();
ShutdownSUBTRANS();
ShutdownMultiXact();
@ -5395,12 +5424,21 @@ ShutdownXLOG(int code, Datum arg)
/*
* Perform a checkpoint --- either during shutdown, or on-the-fly
*
* If force is true, we force a checkpoint regardless of whether any XLOG
* activity has occurred since the last one.
* flags is a bitwise OR of the following:
* CHECKPOINT_IS_SHUTDOWN: checkpoint is for database shutdown.
* CHECKPOINT_IMMEDIATE: finish the checkpoint ASAP,
* ignoring checkpoint_completion_target parameter.
* CHECKPOINT_FORCE: force a checkpoint even if no XLOG activity has occured
* since the last one (implied by CHECKPOINT_IS_SHUTDOWN).
*
* Note: flags might contain other bits of interest to RequestCheckpoint.
* In particular note that this routine is synchronous and does not pay
* attention to CHECKPOINT_WAIT.
*/
void
CreateCheckPoint(bool shutdown, bool force)
CreateCheckPoint(int flags)
{
bool shutdown = (flags & CHECKPOINT_IS_SHUTDOWN) != 0;
CheckPoint checkPoint;
XLogRecPtr recptr;
XLogCtlInsert *Insert = &XLogCtl->Insert;
@ -5459,7 +5497,7 @@ CreateCheckPoint(bool shutdown, bool force)
* the end of the last checkpoint record, and its redo pointer must point
* to itself.
*/
if (!shutdown && !force)
if ((flags & (CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_FORCE)) == 0)
{
XLogRecPtr curInsert;
@ -5591,7 +5629,7 @@ CreateCheckPoint(bool shutdown, bool force)
*/
END_CRIT_SECTION();
CheckPointGuts(checkPoint.redo);
CheckPointGuts(checkPoint.redo, flags);
START_CRIT_SECTION();
@ -5650,24 +5688,24 @@ CreateCheckPoint(bool shutdown, bool force)
/*
* We are now done with critical updates; no need for system panic if we
* have trouble while fooling with offline log segments.
* have trouble while fooling with old log segments.
*/
END_CRIT_SECTION();
/*
* Delete offline log files (those no longer needed even for previous
* Delete old log files (those no longer needed even for previous
* checkpoint).
*/
if (_logId || _logSeg)
{
PrevLogSeg(_logId, _logSeg);
MoveOfflineLogs(_logId, _logSeg, recptr,
&nsegsremoved, &nsegsrecycled);
RemoveOldXlogFiles(_logId, _logSeg, recptr,
&nsegsremoved, &nsegsrecycled);
}
/*
* Make more log segments if needed. (Do this after deleting offline log
* segments, to avoid having peak disk space usage higher than necessary.)
* Make more log segments if needed. (Do this after recycling old log
* segments, since that may supply some of the needed files.)
*/
if (!shutdown)
nsegsadded = PreallocXlogFiles(recptr);
@ -5697,12 +5735,12 @@ CreateCheckPoint(bool shutdown, bool force)
* recovery restartpoints.
*/
static void
CheckPointGuts(XLogRecPtr checkPointRedo)
CheckPointGuts(XLogRecPtr checkPointRedo, int flags)
{
CheckPointCLOG();
CheckPointSUBTRANS();
CheckPointMultiXact();
FlushBufferPool(); /* performs all required fsyncs */
CheckPointBuffers(flags); /* performs all required fsyncs */
/* We deliberately delay 2PC checkpointing as long as possible */
CheckPointTwoPhase(checkPointRedo);
}
@ -5710,7 +5748,7 @@ CheckPointGuts(XLogRecPtr checkPointRedo)
/*
* Set a recovery restart point if appropriate
*
* This is similar to CreateCheckpoint, but is used during WAL recovery
* This is similar to CreateCheckPoint, but is used during WAL recovery
* to establish a point from which recovery can roll forward without
* replaying the entire recovery log. This function is called each time
* a checkpoint record is read from XLOG; it must determine whether a
@ -5751,7 +5789,7 @@ RecoveryRestartPoint(const CheckPoint *checkPoint)
/*
* OK, force data out to disk
*/
CheckPointGuts(checkPoint->redo);
CheckPointGuts(checkPoint->redo, CHECKPOINT_IMMEDIATE);
/*
* Update pg_control so that any subsequent crash will restart from this
@ -6176,8 +6214,10 @@ pg_start_backup(PG_FUNCTION_ARGS)
* page problems, this guarantees that two successive backup runs will
* have different checkpoint positions and hence different history
* file names, even if nothing happened in between.
*
* We don't use CHECKPOINT_IMMEDIATE, hence this can take awhile.
*/
RequestCheckpoint(true, false);
RequestCheckpoint(CHECKPOINT_FORCE | CHECKPOINT_WAIT);
/*
* Now we need to fetch the checkpoint record location, and also its

View File

@ -8,7 +8,7 @@
* Portions Copyright (c) 1994, Regents of the University of California
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/bootstrap/bootstrap.c,v 1.233 2007/03/07 13:35:02 alvherre Exp $
* $PostgreSQL: pgsql/src/backend/bootstrap/bootstrap.c,v 1.234 2007/06/28 00:02:37 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -489,7 +489,7 @@ BootstrapModeMain(void)
/* Perform a checkpoint to ensure everything's down to disk */
SetProcessingMode(NormalProcessing);
CreateCheckPoint(true, true);
CreateCheckPoint(CHECKPOINT_IS_SHUTDOWN | CHECKPOINT_IMMEDIATE);
/* Clean up and exit */
cleanup();

View File

@ -3,7 +3,7 @@
*
* Copyright (c) 1996-2007, PostgreSQL Global Development Group
*
* $PostgreSQL: pgsql/src/backend/catalog/system_views.sql,v 1.37 2007/03/30 18:34:55 mha Exp $
* $PostgreSQL: pgsql/src/backend/catalog/system_views.sql,v 1.38 2007/06/28 00:02:37 tgl Exp $
*/
CREATE VIEW pg_roles AS
@ -366,11 +366,9 @@ CREATE VIEW pg_stat_database AS
FROM pg_database D;
CREATE VIEW pg_stat_bgwriter AS
SELECT
pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints_timed,
pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req,
pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint,
pg_stat_get_bgwriter_buf_written_lru() AS buffers_lru,
pg_stat_get_bgwriter_buf_written_all() AS buffers_all,
pg_stat_get_bgwriter_maxwritten_lru() AS maxwritten_lru,
pg_stat_get_bgwriter_maxwritten_all() AS maxwritten_all;
SELECT
pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints_timed,
pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req,
pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint,
pg_stat_get_bgwriter_buf_written_clean() AS buffers_clean,
pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean;

View File

@ -13,7 +13,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/commands/dbcommands.c,v 1.195 2007/06/01 19:38:07 tgl Exp $
* $PostgreSQL: pgsql/src/backend/commands/dbcommands.c,v 1.196 2007/06/28 00:02:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -401,10 +401,9 @@ createdb(const CreatedbStmt *stmt)
/*
* Force dirty buffers out to disk, to ensure source database is
* up-to-date for the copy. (We really only need to flush buffers for the
* source database, but bufmgr.c provides no API for that.)
* up-to-date for the copy.
*/
BufferSync();
FlushDatabaseBuffers(src_dboid);
/*
* Once we start copying subdirectories, we need to be able to clean 'em
@ -507,7 +506,7 @@ createdb(const CreatedbStmt *stmt)
* Perhaps if we ever implement CREATE DATABASE in a less cheesy way,
* we can avoid this.
*/
RequestCheckpoint(true, false);
RequestCheckpoint(CHECKPOINT_IMMEDIATE | CHECKPOINT_FORCE | CHECKPOINT_WAIT);
/*
* Close pg_database, but keep lock till commit (this is important to
@ -661,7 +660,7 @@ dropdb(const char *dbname, bool missing_ok)
* open files, which would cause rmdir() to fail.
*/
#ifdef WIN32
RequestCheckpoint(true, false);
RequestCheckpoint(CHECKPOINT_IMMEDIATE | CHECKPOINT_FORCE | CHECKPOINT_WAIT);
#endif
/*
@ -1424,10 +1423,9 @@ dbase_redo(XLogRecPtr lsn, XLogRecord *record)
/*
* Force dirty buffers out to disk, to ensure source database is
* up-to-date for the copy. (We really only need to flush buffers for
* the source database, but bufmgr.c provides no API for that.)
* up-to-date for the copy.
*/
BufferSync();
FlushDatabaseBuffers(xlrec->src_db_id);
/*
* Copy this subdirectory to the new location

View File

@ -37,13 +37,14 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/postmaster/bgwriter.c,v 1.38 2007/05/27 03:50:39 tgl Exp $
* $PostgreSQL: pgsql/src/backend/postmaster/bgwriter.c,v 1.39 2007/06/28 00:02:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
#include "postgres.h"
#include <signal.h>
#include <sys/time.h>
#include <time.h>
#include <unistd.h>
@ -59,6 +60,7 @@
#include "storage/pmsignal.h"
#include "storage/shmem.h"
#include "storage/smgr.h"
#include "storage/spin.h"
#include "tcop/tcopprot.h"
#include "utils/guc.h"
#include "utils/memutils.h"
@ -70,19 +72,20 @@
*
* 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, bgwriter increments ckpt_started.
* * At start of a checkpoint, bgwriter reads (and clears) the request flags
* and increments ckpt_started, while holding ckpt_lck.
* * On completion of a checkpoint, bgwriter sets ckpt_done to
* equal ckpt_started.
* * On failure of a checkpoint, bgwrite first increments ckpt_failed,
* then sets ckpt_done to equal ckpt_started.
* All three fields are declared sig_atomic_t to ensure they can be read
* and written without explicit locking. The algorithm for backends is:
* 1. Record current values of ckpt_failed and ckpt_started (in that
* order!).
* * On failure of a checkpoint, bgwriter 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).
* 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
@ -91,10 +94,9 @@
* 6. If ckpt_failed is different from the originally saved value,
* assume request failed; otherwise it was definitely successful.
*
* An additional field is ckpt_time_warn; this is also sig_atomic_t for
* simplicity, but is only used as a boolean. If a backend is requesting
* a checkpoint for which a checkpoints-too-close-together warning is
* reasonable, it should set this field TRUE just before sending the signal.
* 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 bgwriter. Unlike the checkpoint fields, the requests
@ -112,11 +114,13 @@ typedef struct
{
pid_t bgwriter_pid; /* PID of bgwriter (0 if not started) */
sig_atomic_t ckpt_started; /* advances when checkpoint starts */
sig_atomic_t ckpt_done; /* advances when checkpoint done */
sig_atomic_t ckpt_failed; /* advances when checkpoint fails */
slock_t ckpt_lck; /* protects all the ckpt_* fields */
sig_atomic_t ckpt_time_warn; /* warn if too soon since last ckpt? */
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 */
int num_requests; /* current # of requests */
int max_requests; /* allocated array size */
@ -125,12 +129,16 @@ typedef struct
static BgWriterShmemStruct *BgWriterShmem;
/* interval for calling AbsorbFsyncRequests in CheckpointWriteDelay */
#define WRITES_PER_ABSORB 1000
/*
* GUC parameters
*/
int BgWriterDelay = 200;
int CheckPointTimeout = 300;
int CheckPointWarning = 30;
double CheckPointCompletionTarget = 0.5;
/*
* Flags set by interrupt handlers for later service in the main loop.
@ -146,9 +154,22 @@ static bool am_bg_writer = false;
static bool ckpt_active = false;
/* these values are valid when ckpt_active is true: */
static time_t ckpt_start_time;
static XLogRecPtr ckpt_start_recptr;
static double ckpt_cached_elapsed;
static time_t last_checkpoint_time;
static time_t last_xlog_switch_time;
/* Prototypes for private functions */
static void CheckArchiveTimeout(void);
static void BgWriterNap(void);
static bool IsCheckpointOnSchedule(double progress);
static bool ImmediateCheckpointRequested(void);
/* Signal handlers */
static void bg_quickdie(SIGNAL_ARGS);
static void BgSigHupHandler(SIGNAL_ARGS);
@ -281,8 +302,11 @@ BackgroundWriterMain(void)
/* use volatile pointer to prevent code rearrangement */
volatile BgWriterShmemStruct *bgs = BgWriterShmem;
SpinLockAcquire(&bgs->ckpt_lck);
bgs->ckpt_failed++;
bgs->ckpt_done = bgs->ckpt_started;
SpinLockRelease(&bgs->ckpt_lck);
ckpt_active = false;
}
@ -328,10 +352,8 @@ BackgroundWriterMain(void)
for (;;)
{
bool do_checkpoint = false;
bool force_checkpoint = false;
time_t now;
int elapsed_secs;
long udelay;
/*
* Emergency bailout if postmaster has died. This is to avoid the
@ -354,7 +376,6 @@ BackgroundWriterMain(void)
{
checkpoint_requested = false;
do_checkpoint = true;
force_checkpoint = true;
BgWriterStats.m_requested_checkpoints++;
}
if (shutdown_requested)
@ -377,11 +398,10 @@ BackgroundWriterMain(void)
*/
now = time(NULL);
elapsed_secs = now - last_checkpoint_time;
if (elapsed_secs >= CheckPointTimeout)
if (!do_checkpoint && elapsed_secs >= CheckPointTimeout)
{
do_checkpoint = true;
if (!force_checkpoint)
BgWriterStats.m_timed_checkpoints++;
BgWriterStats.m_timed_checkpoints++;
}
/*
@ -390,28 +410,48 @@ BackgroundWriterMain(void)
*/
if (do_checkpoint)
{
/* use volatile pointer to prevent code rearrangement */
volatile BgWriterShmemStruct *bgs = BgWriterShmem;
int flags;
/*
* 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(&bgs->ckpt_lck);
flags = bgs->ckpt_flags;
bgs->ckpt_flags = 0;
bgs->ckpt_started++;
SpinLockRelease(&bgs->ckpt_lck);
/*
* We will warn if (a) too soon since last checkpoint (whatever
* caused it) and (b) somebody has set the ckpt_time_warn flag
* caused it) and (b) somebody set the CHECKPOINT_WARNONTIME flag
* since the last checkpoint start. Note in particular that this
* implementation will not generate warnings caused by
* CheckPointTimeout < CheckPointWarning.
*/
if (BgWriterShmem->ckpt_time_warn &&
if ((flags & CHECKPOINT_WARNONTIME) &&
elapsed_secs < CheckPointWarning)
ereport(LOG,
(errmsg("checkpoints are occurring too frequently (%d seconds apart)",
elapsed_secs),
errhint("Consider increasing the configuration parameter \"checkpoint_segments\".")));
BgWriterShmem->ckpt_time_warn = false;
/*
* Indicate checkpoint start to any waiting backends.
* Initialize bgwriter-private variables used during checkpoint.
*/
ckpt_active = true;
BgWriterShmem->ckpt_started++;
ckpt_start_recptr = GetInsertRecPtr();
ckpt_start_time = now;
ckpt_cached_elapsed = 0;
CreateCheckPoint(false, force_checkpoint);
/*
* Do the checkpoint.
*/
CreateCheckPoint(flags);
/*
* After any checkpoint, close all smgr files. This is so we
@ -422,7 +462,10 @@ BackgroundWriterMain(void)
/*
* Indicate checkpoint completion to any waiting backends.
*/
BgWriterShmem->ckpt_done = BgWriterShmem->ckpt_started;
SpinLockAcquire(&bgs->ckpt_lck);
bgs->ckpt_done = bgs->ckpt_started;
SpinLockRelease(&bgs->ckpt_lck);
ckpt_active = false;
/*
@ -435,88 +478,260 @@ BackgroundWriterMain(void)
else
BgBufferSync();
/*
* Check for archive_timeout, if so, switch xlog files. First we do a
* quick check using possibly-stale local state.
*/
if (XLogArchiveTimeout > 0 &&
(int) (now - last_xlog_switch_time) >= XLogArchiveTimeout)
{
/*
* Update local state ... note that last_xlog_switch_time is the
* last time a switch was performed *or requested*.
*/
time_t last_time = GetLastSegSwitchTime();
/* Check for archive_timeout and switch xlog files if necessary. */
CheckArchiveTimeout();
last_xlog_switch_time = Max(last_xlog_switch_time, last_time);
/* if we did a checkpoint, 'now' might be stale too */
if (do_checkpoint)
now = time(NULL);
/* Now we can do the real check */
if ((int) (now - last_xlog_switch_time) >= XLogArchiveTimeout)
{
XLogRecPtr switchpoint;
/* OK, it's time to switch */
switchpoint = RequestXLogSwitch();
/*
* If the returned pointer points exactly to a segment
* boundary, assume nothing happened.
*/
if ((switchpoint.xrecoff % XLogSegSize) != 0)
ereport(DEBUG1,
(errmsg("transaction log switch forced (archive_timeout=%d)",
XLogArchiveTimeout)));
/*
* Update state in any case, so we don't retry constantly when
* the system is idle.
*/
last_xlog_switch_time = now;
}
}
/*
* Send off activity statistics to the stats collector
*/
pgstat_send_bgwriter();
/*
* Nap for the configured time, or sleep for 10 seconds if there is no
* bgwriter activity configured.
*
* On some platforms, signals won't interrupt the sleep. To ensure we
* respond reasonably promptly when someone signals us, break down the
* sleep into 1-second increments, and check for interrupts after each
* nap.
*
* We absorb pending requests after each short sleep.
*/
if ((bgwriter_all_percent > 0.0 && bgwriter_all_maxpages > 0) ||
(bgwriter_lru_percent > 0.0 && bgwriter_lru_maxpages > 0))
udelay = BgWriterDelay * 1000L;
else if (XLogArchiveTimeout > 0)
udelay = 1000000L; /* One second */
else
udelay = 10000000L; /* Ten seconds */
while (udelay > 999999L)
{
if (got_SIGHUP || checkpoint_requested || shutdown_requested)
break;
pg_usleep(1000000L);
AbsorbFsyncRequests();
udelay -= 1000000L;
}
if (!(got_SIGHUP || checkpoint_requested || shutdown_requested))
pg_usleep(udelay);
/* Nap for the configured time. */
BgWriterNap();
}
}
/*
* CheckArchiveTimeout -- check for archive_timeout and switch xlog files
* if needed
*/
static void
CheckArchiveTimeout(void)
{
time_t now;
time_t last_time;
if (XLogArchiveTimeout <= 0)
return;
now = 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 = GetLastSegSwitchTime();
last_xlog_switch_time = Max(last_xlog_switch_time, last_time);
/* Now we can do the real check */
if ((int) (now - last_xlog_switch_time) >= XLogArchiveTimeout)
{
XLogRecPtr switchpoint;
/* OK, it's time to switch */
switchpoint = RequestXLogSwitch();
/*
* If the returned pointer points exactly to a segment
* boundary, assume nothing happened.
*/
if ((switchpoint.xrecoff % XLogSegSize) != 0)
ereport(DEBUG1,
(errmsg("transaction log switch forced (archive_timeout=%d)",
XLogArchiveTimeout)));
/*
* Update state in any case, so we don't retry constantly when
* the system is idle.
*/
last_xlog_switch_time = now;
}
}
/*
* BgWriterNap -- Nap for the configured time or until a signal is received.
*/
static void
BgWriterNap(void)
{
long udelay;
/*
* Send off activity statistics to the stats collector
*/
pgstat_send_bgwriter();
/*
* Nap for the configured time, or sleep for 10 seconds if there is no
* bgwriter activity configured.
*
* On some platforms, signals won't interrupt the sleep. To ensure we
* respond reasonably promptly when someone signals us, break down the
* sleep into 1-second increments, and check for interrupts after each
* nap.
*
* We absorb pending requests after each short sleep.
*/
if ((bgwriter_lru_percent > 0.0 && bgwriter_lru_maxpages > 0) ||
ckpt_active)
udelay = BgWriterDelay * 1000L;
else if (XLogArchiveTimeout > 0)
udelay = 1000000L; /* One second */
else
udelay = 10000000L; /* Ten seconds */
while (udelay > 999999L)
{
if (got_SIGHUP || shutdown_requested ||
(ckpt_active ? ImmediateCheckpointRequested() : checkpoint_requested))
break;
pg_usleep(1000000L);
AbsorbFsyncRequests();
udelay -= 1000000L;
}
if (!(got_SIGHUP || shutdown_requested ||
(ckpt_active ? ImmediateCheckpointRequested() : checkpoint_requested)))
pg_usleep(udelay);
}
/*
* 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)
{
if (checkpoint_requested)
{
volatile BgWriterShmemStruct *bgs = BgWriterShmem;
/*
* We don't need to acquire the ckpt_lck in this case because we're
* only looking at a single flag bit.
*/
if (bgs->ckpt_flags & CHECKPOINT_IMMEDIATE)
return true;
}
return false;
}
/*
* CheckpointWriteDelay -- yield control to bgwriter during a checkpoint
*
* This function is called after each page write performed by BufferSync().
* It is responsible for keeping the bgwriter's normal activities in
* progress during a long checkpoint, and 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-bgwriter process */
if (!am_bg_writer)
return;
/*
* Perform the usual bgwriter 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) &&
!shutdown_requested &&
!ImmediateCheckpointRequested() &&
IsCheckpointOnSchedule(progress))
{
if (got_SIGHUP)
{
got_SIGHUP = false;
ProcessConfigFile(PGC_SIGHUP);
}
BgBufferSync();
CheckArchiveTimeout();
BgWriterNap();
AbsorbFsyncRequests();
absorb_counter = WRITES_PER_ABSORB;
}
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.
*/
AbsorbFsyncRequests();
absorb_counter = WRITES_PER_ABSORB;
}
}
/*
* IsCheckpointOnSchedule -- are we on schedule to finish this checkpoint
* in time?
*
* 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 checkpoint_segments.
*
* We compare the current WAL insert location against the location
* computed before calling CreateCheckPoint. The code in XLogInsert that
* actually triggers a checkpoint when checkpoint_segments is exceeded
* compares against RedoRecptr, so this is not completely accurate.
* However, it's good enough for our purposes, we're only calculating
* an estimate anyway.
*/
recptr = GetInsertRecPtr();
elapsed_xlogs =
(((double) (int32) (recptr.xlogid - ckpt_start_recptr.xlogid)) * XLogSegsPerFile +
((double) (int32) (recptr.xrecoff - ckpt_start_recptr.xrecoff)) / XLogSegSize) /
CheckPointSegments;
if (progress < elapsed_xlogs)
{
ckpt_cached_elapsed = elapsed_xlogs;
return false;
}
/*
* Check progress against time elapsed and checkpoint_timeout.
*/
gettimeofday(&now, NULL);
elapsed_time = ((double) (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
@ -614,35 +829,45 @@ BgWriterShmemInit(void)
return; /* already initialized */
MemSet(BgWriterShmem, 0, sizeof(BgWriterShmemStruct));
SpinLockInit(&BgWriterShmem->ckpt_lck);
BgWriterShmem->max_requests = NBuffers;
}
/*
* RequestCheckpoint
* Called in backend processes to request an immediate checkpoint
* Called in backend processes to request a checkpoint
*
* If waitforit is true, wait until the checkpoint is completed
* before returning; otherwise, just signal the request and return
* immediately.
*
* If warnontime is true, and it's "too soon" since the last checkpoint,
* the bgwriter will log a warning. This should be true only for checkpoints
* caused due to xlog filling, else the warning will be misleading.
* flags is a bitwise OR of the following:
* CHECKPOINT_IS_SHUTDOWN: checkpoint is for database shutdown.
* CHECKPOINT_IMMEDIATE: finish the checkpoint ASAP,
* ignoring checkpoint_completion_target parameter.
* CHECKPOINT_FORCE: force a checkpoint even if no XLOG activity has occured
* since the last one (implied by CHECKPOINT_IS_SHUTDOWN).
* CHECKPOINT_WARNONTIME: if it's "too soon" since the last checkpoint,
* the bgwriter will log a warning. This should be true only for
* checkpoints requested due to xlog filling, else the warning will
* be misleading.
* CHECKPOINT_WAIT: wait for completion before returning (otherwise,
* just signal bgwriter to do it, and return).
*/
void
RequestCheckpoint(bool waitforit, bool warnontime)
RequestCheckpoint(int flags)
{
/* use volatile pointer to prevent code rearrangement */
volatile BgWriterShmemStruct *bgs = BgWriterShmem;
sig_atomic_t old_failed = bgs->ckpt_failed;
sig_atomic_t old_started = bgs->ckpt_started;
int old_failed, old_started;
/*
* If in a standalone backend, just do it ourselves.
*/
if (!IsPostmasterEnvironment)
{
CreateCheckPoint(false, true);
/*
* 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);
/*
* After any checkpoint, close all smgr files. This is so we won't
@ -653,49 +878,76 @@ RequestCheckpoint(bool waitforit, bool warnontime)
return;
}
/* Set warning request flag if appropriate */
if (warnontime)
bgs->ckpt_time_warn = true;
/*
* 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 bgwriter.
*
* 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(&bgs->ckpt_lck);
old_failed = bgs->ckpt_failed;
old_started = bgs->ckpt_started;
bgs->ckpt_flags |= (flags & ~CHECKPOINT_WAIT);
SpinLockRelease(&bgs->ckpt_lck);
/*
* Send signal to request checkpoint. When waitforit is false, we
* Send signal to request checkpoint. When not waiting, we
* consider failure to send the signal to be nonfatal.
*/
if (BgWriterShmem->bgwriter_pid == 0)
elog(waitforit ? ERROR : LOG,
elog((flags & CHECKPOINT_WAIT) ? ERROR : LOG,
"could not request checkpoint because bgwriter not running");
if (kill(BgWriterShmem->bgwriter_pid, SIGINT) != 0)
elog(waitforit ? ERROR : LOG,
elog((flags & CHECKPOINT_WAIT) ? ERROR : LOG,
"could not signal for checkpoint: %m");
/*
* If requested, wait for completion. We detect completion according to
* the algorithm given above.
*/
if (waitforit)
if (flags & CHECKPOINT_WAIT)
{
while (bgs->ckpt_started == old_started)
int new_started, new_failed;
/* Wait for a new checkpoint to start. */
for(;;)
{
SpinLockAcquire(&bgs->ckpt_lck);
new_started = bgs->ckpt_started;
SpinLockRelease(&bgs->ckpt_lck);
if (new_started != old_started)
break;
CHECK_FOR_INTERRUPTS();
pg_usleep(100000L);
}
old_started = bgs->ckpt_started;
/*
* We are waiting for ckpt_done >= old_started, in a modulo sense.
* This is a little tricky since we don't know the width or signedness
* of sig_atomic_t. We make the lowest common denominator assumption
* that it is only as wide as "char". This means that this algorithm
* will cope correctly as long as we don't sleep for more than 127
* completed checkpoints. (If we do, we will get another chance to
* exit after 128 more checkpoints...)
* We are waiting for ckpt_done >= new_started, in a modulo sense.
*/
while (((signed char) (bgs->ckpt_done - old_started)) < 0)
for(;;)
{
int new_done;
SpinLockAcquire(&bgs->ckpt_lck);
new_done = bgs->ckpt_done;
new_failed = bgs->ckpt_failed;
SpinLockRelease(&bgs->ckpt_lck);
if (new_done - new_started >= 0)
break;
CHECK_FOR_INTERRUPTS();
pg_usleep(100000L);
}
if (bgs->ckpt_failed != old_failed)
if (new_failed != old_failed)
ereport(ERROR,
(errmsg("checkpoint request failed"),
errhint("Consult recent messages in the server log for details.")));

View File

@ -13,7 +13,7 @@
*
* Copyright (c) 2001-2007, PostgreSQL Global Development Group
*
* $PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.159 2007/06/07 18:53:17 alvherre Exp $
* $PostgreSQL: pgsql/src/backend/postmaster/pgstat.c,v 1.160 2007/06/28 00:02:38 tgl Exp $
* ----------
*/
#include "postgres.h"
@ -3174,8 +3174,6 @@ pgstat_recv_bgwriter(PgStat_MsgBgWriter *msg, int len)
globalStats.timed_checkpoints += msg->m_timed_checkpoints;
globalStats.requested_checkpoints += msg->m_requested_checkpoints;
globalStats.buf_written_checkpoints += msg->m_buf_written_checkpoints;
globalStats.buf_written_lru += msg->m_buf_written_lru;
globalStats.buf_written_all += msg->m_buf_written_all;
globalStats.maxwritten_lru += msg->m_maxwritten_lru;
globalStats.maxwritten_all += msg->m_maxwritten_all;
globalStats.buf_written_clean += msg->m_buf_written_clean;
globalStats.maxwritten_clean += msg->m_maxwritten_clean;
}

View File

@ -8,35 +8,23 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.221 2007/06/18 00:47:20 tgl Exp $
* $PostgreSQL: pgsql/src/backend/storage/buffer/bufmgr.c,v 1.222 2007/06/28 00:02:38 tgl Exp $
*
*-------------------------------------------------------------------------
*/
/*
* Principal entry points:
*
* ReadBuffer() -- find or create a buffer holding the requested page,
* and pin it so that no one can destroy it while this process
* is using it.
*
* ReadOrZeroBuffer() -- like ReadBuffer, but if the page is not already in
* cache we don't read it, but just return a zeroed-out buffer. Useful
* when the caller intends to fill the page from scratch, since this
* saves I/O and avoids unnecessary failure if the page-on-disk has
* corrupt page headers. Caution: do not use this to read a page that
* is beyond the relation's current physical EOF; that is likely to
* cause problems in md.c when the page is modified and written out.
*
* ReleaseBuffer() -- unpin a buffer
*
* MarkBufferDirty() -- mark a pinned buffer's contents as "dirty".
* The disk write is delayed until buffer replacement or checkpoint.
*
* BufferSync() -- flush all dirty buffers in the buffer pool.
*
* BgBufferSync() -- flush some dirty buffers in the buffer pool.
*
* InitBufferPool() -- Init the buffer module.
*
* See other files:
* See also these files:
* freelist.c -- chooses victim for buffer replacement
* buf_table.c -- manages the buffer lookup table
*/
@ -64,16 +52,11 @@
#define LocalBufHdrGetBlock(bufHdr) \
LocalBufferBlockPointers[-((bufHdr)->buf_id + 2)]
/* interval for calling AbsorbFsyncRequests in BufferSync */
#define WRITES_PER_ABSORB 1000
/* GUC variables */
bool zero_damaged_pages = false;
double bgwriter_lru_percent = 1.0;
double bgwriter_all_percent = 0.333;
int bgwriter_lru_maxpages = 5;
int bgwriter_all_maxpages = 5;
long NDirectFileRead; /* some I/O's are direct file access. bypass
@ -95,6 +78,7 @@ static Buffer ReadBuffer_common(Relation reln, BlockNumber blockNum,
static bool PinBuffer(volatile BufferDesc *buf, BufferAccessStrategy strategy);
static void PinBuffer_Locked(volatile BufferDesc *buf);
static void UnpinBuffer(volatile BufferDesc *buf, bool fixOwner);
static void BufferSync(int flags);
static bool SyncOneBuffer(int buf_id, bool skip_pinned);
static void WaitIO(volatile BufferDesc *buf);
static bool StartBufferIO(volatile BufferDesc *buf, bool forInput);
@ -143,8 +127,10 @@ ReadBufferWithStrategy(Relation reln, BlockNumber blockNum,
/*
* ReadOrZeroBuffer -- like ReadBuffer, but if the page isn't in buffer
* cache already, it's filled with zeros instead of reading it from
* disk. The caller is expected to overwrite the whole buffer,
* so that the current page contents are not interesting.
* disk. Useful when the caller intends to fill the page from scratch,
* since this saves I/O and avoids unnecessary failure if the
* page-on-disk has corrupt page headers.
*
* Caution: do not use this to read a page that is beyond the relation's
* current physical EOF; that is likely to cause problems in md.c when
* the page is modified and written out. P_NEW is OK, though.
@ -644,7 +630,7 @@ BufferAlloc(Relation reln,
* at 1 so that the buffer can survive one clock-sweep pass.)
*/
buf->tag = newTag;
buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_IO_ERROR);
buf->flags &= ~(BM_VALID | BM_DIRTY | BM_JUST_DIRTIED | BM_CHECKPOINT_NEEDED | BM_IO_ERROR);
buf->flags |= BM_TAG_VALID;
buf->usage_count = 1;
@ -999,45 +985,114 @@ UnpinBuffer(volatile BufferDesc *buf, bool fixOwner)
* BufferSync -- Write out all dirty buffers in the pool.
*
* This is called at checkpoint time to write out all dirty shared buffers.
* The checkpoint request flags should be passed in; currently the only one
* examined is CHECKPOINT_IMMEDIATE, which disables delays between writes.
*/
void
BufferSync(void)
static void
BufferSync(int flags)
{
int buf_id;
int num_to_scan;
int absorb_counter;
/*
* Find out where to start the circular scan.
*/
buf_id = StrategySyncStart();
int num_to_write;
int num_written;
/* Make sure we can handle the pin inside SyncOneBuffer */
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
/*
* Loop over all buffers.
* Loop over all buffers, and mark the ones that need to be written with
* BM_CHECKPOINT_NEEDED. Count them as we go (num_to_write), so that we
* can estimate how much work needs to be done.
*
* This allows us to write only those pages that were dirty when the
* checkpoint began, and not those that get dirtied while it proceeds.
* Whenever a page with BM_CHECKPOINT_NEEDED is written out, either by us
* later in this function, or by normal backends or the bgwriter cleaning
* scan, the flag is cleared. Any buffer dirtied after this point won't
* have the flag set.
*
* Note that if we fail to write some buffer, we may leave buffers with
* BM_CHECKPOINT_NEEDED still set. This is OK since any such buffer
* would certainly need to be written for the next checkpoint attempt,
* too.
*/
num_to_write = 0;
for (buf_id = 0; buf_id < NBuffers; buf_id++)
{
volatile BufferDesc *bufHdr = &BufferDescriptors[buf_id];
/*
* Header spinlock is enough to examine BM_DIRTY, see comment in
* SyncOneBuffer.
*/
LockBufHdr(bufHdr);
if (bufHdr->flags & BM_DIRTY)
{
bufHdr->flags |= BM_CHECKPOINT_NEEDED;
num_to_write++;
}
UnlockBufHdr(bufHdr);
}
if (num_to_write == 0)
return; /* nothing to do */
/*
* Loop over all buffers again, and write the ones (still) marked with
* BM_CHECKPOINT_NEEDED. In this loop, we start at the clock sweep
* point since we might as well dump soon-to-be-recycled buffers first.
*/
buf_id = StrategySyncStart();
num_to_scan = NBuffers;
absorb_counter = WRITES_PER_ABSORB;
num_written = 0;
while (num_to_scan-- > 0)
{
if (SyncOneBuffer(buf_id, false))
{
BgWriterStats.m_buf_written_checkpoints++;
volatile BufferDesc *bufHdr = &BufferDescriptors[buf_id];
/*
* If in bgwriter, absorb pending fsync requests after each
* WRITES_PER_ABSORB write operations, to prevent overflow of the
* fsync request queue. If not in bgwriter process, this is a
* no-op.
*/
if (--absorb_counter <= 0)
/*
* We don't need to acquire the lock here, because we're only looking
* at a single bit. It's possible that someone else writes the buffer
* and clears the flag right after we check, but that doesn't matter
* since SyncOneBuffer will then do nothing. However, there is a
* further race condition: it's conceivable that between the time we
* examine the bit here and the time SyncOneBuffer acquires lock,
* someone else not only wrote the buffer but replaced it with another
* page and dirtied it. In that improbable case, SyncOneBuffer will
* write the buffer though we didn't need to. It doesn't seem
* worth guarding against this, though.
*/
if (bufHdr->flags & BM_CHECKPOINT_NEEDED)
{
if (SyncOneBuffer(buf_id, false))
{
AbsorbFsyncRequests();
absorb_counter = WRITES_PER_ABSORB;
BgWriterStats.m_buf_written_checkpoints++;
num_written++;
/*
* We know there are at most num_to_write buffers with
* BM_CHECKPOINT_NEEDED set; so we can stop scanning if
* num_written reaches num_to_write.
*
* Note that num_written doesn't include buffers written by
* other backends, or by the bgwriter cleaning scan. That
* means that the estimate of how much progress we've made is
* conservative, and also that this test will often fail to
* trigger. But it seems worth making anyway.
*/
if (num_written >= num_to_write)
break;
/*
* Perform normal bgwriter duties and sleep to throttle
* our I/O rate.
*/
CheckpointWriteDelay(flags,
(double) num_written / num_to_write);
}
}
if (++buf_id >= NBuffers)
buf_id = 0;
}
@ -1051,8 +1106,7 @@ BufferSync(void)
void
BgBufferSync(void)
{
static int buf_id1 = 0;
int buf_id2;
int buf_id;
int num_to_scan;
int num_written;
@ -1060,45 +1114,10 @@ BgBufferSync(void)
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
/*
* To minimize work at checkpoint time, we want to try to keep all the
* buffers clean; this motivates a scan that proceeds sequentially through
* all buffers. But we are also charged with ensuring that buffers that
* The purpose of this sweep is to ensure that buffers that
* will be recycled soon are clean when needed; these buffers are the ones
* just ahead of the StrategySyncStart point. We make a separate scan
* through those.
*/
/*
* This loop runs over all buffers, including pinned ones. The starting
* point advances through the buffer pool on successive calls.
* just ahead of the StrategySyncStart point.
*
* Note that we advance the static counter *before* trying to write. This
* ensures that, if we have a persistent write failure on a dirty buffer,
* we'll still be able to make progress writing other buffers. (The
* bgwriter will catch the error and just call us again later.)
*/
if (bgwriter_all_percent > 0.0 && bgwriter_all_maxpages > 0)
{
num_to_scan = (int) ((NBuffers * bgwriter_all_percent + 99) / 100);
num_written = 0;
while (num_to_scan-- > 0)
{
if (++buf_id1 >= NBuffers)
buf_id1 = 0;
if (SyncOneBuffer(buf_id1, false))
{
if (++num_written >= bgwriter_all_maxpages)
{
BgWriterStats.m_maxwritten_all++;
break;
}
}
}
BgWriterStats.m_buf_written_all += num_written;
}
/*
* This loop considers only unpinned buffers close to the clock sweep
* point.
*/
@ -1107,22 +1126,22 @@ BgBufferSync(void)
num_to_scan = (int) ((NBuffers * bgwriter_lru_percent + 99) / 100);
num_written = 0;
buf_id2 = StrategySyncStart();
buf_id = StrategySyncStart();
while (num_to_scan-- > 0)
{
if (SyncOneBuffer(buf_id2, true))
if (SyncOneBuffer(buf_id, true))
{
if (++num_written >= bgwriter_lru_maxpages)
{
BgWriterStats.m_maxwritten_lru++;
BgWriterStats.m_maxwritten_clean++;
break;
}
}
if (++buf_id2 >= NBuffers)
buf_id2 = 0;
if (++buf_id >= NBuffers)
buf_id = 0;
}
BgWriterStats.m_buf_written_lru += num_written;
BgWriterStats.m_buf_written_clean += num_written;
}
}
@ -1333,16 +1352,17 @@ PrintBufferLeakWarning(Buffer buffer)
}
/*
* FlushBufferPool
* CheckPointBuffers
*
* Flush all dirty blocks in buffer pool to disk at the checkpoint time.
* Local relations do not participate in checkpoints, so they don't need to be
* flushed.
* Flush all dirty blocks in buffer pool to disk at checkpoint time.
*
* Note: temporary relations do not participate in checkpoints, so they don't
* need to be flushed.
*/
void
FlushBufferPool(void)
CheckPointBuffers(int flags)
{
BufferSync();
BufferSync(flags);
smgrsync();
}
@ -1732,6 +1752,48 @@ FlushRelationBuffers(Relation rel)
}
}
/* ---------------------------------------------------------------------
* FlushDatabaseBuffers
*
* This function writes all dirty pages of a database out to disk
* (or more accurately, out to kernel disk buffers), ensuring that the
* kernel has an up-to-date view of the database.
*
* Generally, the caller should be holding an appropriate lock to ensure
* no other backend is active in the target database; otherwise more
* pages could get dirtied.
*
* Note we don't worry about flushing any pages of temporary relations.
* It's assumed these wouldn't be interesting.
* --------------------------------------------------------------------
*/
void
FlushDatabaseBuffers(Oid dbid)
{
int i;
volatile BufferDesc *bufHdr;
/* Make sure we can handle the pin inside the loop */
ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
for (i = 0; i < NBuffers; i++)
{
bufHdr = &BufferDescriptors[i];
LockBufHdr(bufHdr);
if (bufHdr->tag.rnode.dbNode == dbid &&
(bufHdr->flags & BM_VALID) && (bufHdr->flags & BM_DIRTY))
{
PinBuffer_Locked(bufHdr);
LWLockAcquire(bufHdr->content_lock, LW_SHARED);
FlushBuffer(bufHdr, NULL);
LWLockRelease(bufHdr->content_lock);
UnpinBuffer(bufHdr, true);
}
else
UnlockBufHdr(bufHdr);
}
}
/*
* ReleaseBuffer -- release the pin on a buffer
*/
@ -2131,7 +2193,7 @@ TerminateBufferIO(volatile BufferDesc *buf, bool clear_dirty,
Assert(buf->flags & BM_IO_IN_PROGRESS);
buf->flags &= ~(BM_IO_IN_PROGRESS | BM_IO_ERROR);
if (clear_dirty && !(buf->flags & BM_JUST_DIRTIED))
buf->flags &= ~BM_DIRTY;
buf->flags &= ~(BM_DIRTY | BM_CHECKPOINT_NEEDED);
buf->flags |= set_flag_bits;
UnlockBufHdr(buf);

View File

@ -10,7 +10,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/tcop/utility.c,v 1.281 2007/06/23 22:12:52 tgl Exp $
* $PostgreSQL: pgsql/src/backend/tcop/utility.c,v 1.282 2007/06/28 00:02:39 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -1152,7 +1152,7 @@ ProcessUtility(Node *parsetree,
ereport(ERROR,
(errcode(ERRCODE_INSUFFICIENT_PRIVILEGE),
errmsg("must be superuser to do CHECKPOINT")));
RequestCheckpoint(true, false);
RequestCheckpoint(CHECKPOINT_IMMEDIATE | CHECKPOINT_FORCE | CHECKPOINT_WAIT);
break;
case T_ReindexStmt:

View File

@ -8,7 +8,7 @@
*
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/adt/pgstatfuncs.c,v 1.42 2007/05/17 23:31:49 tgl Exp $
* $PostgreSQL: pgsql/src/backend/utils/adt/pgstatfuncs.c,v 1.43 2007/06/28 00:02:39 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -64,10 +64,8 @@ extern Datum pg_stat_get_db_tuples_deleted(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_timed_checkpoints(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_requested_checkpoints(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_buf_written_checkpoints(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_buf_written_lru(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_buf_written_all(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_maxwritten_lru(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_maxwritten_all(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_buf_written_clean(PG_FUNCTION_ARGS);
extern Datum pg_stat_get_bgwriter_maxwritten_clean(PG_FUNCTION_ARGS);
extern Datum pg_stat_clear_snapshot(PG_FUNCTION_ARGS);
extern Datum pg_stat_reset(PG_FUNCTION_ARGS);
@ -787,27 +785,15 @@ pg_stat_get_bgwriter_buf_written_checkpoints(PG_FUNCTION_ARGS)
}
Datum
pg_stat_get_bgwriter_buf_written_lru(PG_FUNCTION_ARGS)
pg_stat_get_bgwriter_buf_written_clean(PG_FUNCTION_ARGS)
{
PG_RETURN_INT64(pgstat_fetch_global()->buf_written_lru);
PG_RETURN_INT64(pgstat_fetch_global()->buf_written_clean);
}
Datum
pg_stat_get_bgwriter_buf_written_all(PG_FUNCTION_ARGS)
pg_stat_get_bgwriter_maxwritten_clean(PG_FUNCTION_ARGS)
{
PG_RETURN_INT64(pgstat_fetch_global()->buf_written_all);
}
Datum
pg_stat_get_bgwriter_maxwritten_lru(PG_FUNCTION_ARGS)
{
PG_RETURN_INT64(pgstat_fetch_global()->maxwritten_lru);
}
Datum
pg_stat_get_bgwriter_maxwritten_all(PG_FUNCTION_ARGS)
{
PG_RETURN_INT64(pgstat_fetch_global()->maxwritten_all);
PG_RETURN_INT64(pgstat_fetch_global()->maxwritten_clean);
}

View File

@ -10,7 +10,7 @@
* Written by Peter Eisentraut <peter_e@gmx.net>.
*
* IDENTIFICATION
* $PostgreSQL: pgsql/src/backend/utils/misc/guc.c,v 1.402 2007/06/21 22:59:12 tgl Exp $
* $PostgreSQL: pgsql/src/backend/utils/misc/guc.c,v 1.403 2007/06/28 00:02:39 tgl Exp $
*
*--------------------------------------------------------------------
*/
@ -1573,15 +1573,6 @@ static struct config_int ConfigureNamesInt[] =
5, 0, 1000, NULL, NULL
},
{
{"bgwriter_all_maxpages", PGC_SIGHUP, RESOURCES,
gettext_noop("Background writer maximum number of all pages to flush per round."),
NULL
},
&bgwriter_all_maxpages,
5, 0, 1000, NULL, NULL
},
{
{"log_rotation_age", PGC_SIGHUP, LOGGING_WHERE,
gettext_noop("Automatic log file rotation will occur after N minutes."),
@ -1834,15 +1825,6 @@ static struct config_real ConfigureNamesReal[] =
1.0, 0.0, 100.0, NULL, NULL
},
{
{"bgwriter_all_percent", PGC_SIGHUP, RESOURCES,
gettext_noop("Background writer percentage of all buffers to flush per round."),
NULL
},
&bgwriter_all_percent,
0.333, 0.0, 100.0, NULL, NULL
},
{
{"seed", PGC_USERSET, UNGROUPED,
gettext_noop("Sets the seed for random-number generation."),
@ -1870,6 +1852,15 @@ static struct config_real ConfigureNamesReal[] =
0.1, 0.0, 100.0, NULL, NULL
},
{
{"checkpoint_completion_target", PGC_SIGHUP, WAL_CHECKPOINTS,
gettext_noop("Time spent flushing dirty buffers during checkpoint, as fraction of checkpoint interval."),
NULL
},
&CheckPointCompletionTarget,
0.5, 0.0, 1.0, NULL, NULL
},
/* End-of-list marker */
{
{NULL, 0, 0, NULL, NULL}, NULL, 0.0, 0.0, 0.0, NULL, NULL

View File

@ -140,8 +140,6 @@
#bgwriter_delay = 200ms # 10-10000ms between rounds
#bgwriter_lru_percent = 1.0 # 0-100% of LRU buffers scanned/round
#bgwriter_lru_maxpages = 5 # 0-1000 buffers max written/round
#bgwriter_all_percent = 0.333 # 0-100% of all buffers scanned/round
#bgwriter_all_maxpages = 5 # 0-1000 buffers max written/round
#---------------------------------------------------------------------------
@ -168,6 +166,7 @@
#checkpoint_segments = 3 # in logfile segments, min 1, 16MB each
#checkpoint_timeout = 5min # range 30s-1h
#checkpoint_completion_target = 0.5 # checkpoint target duration, 0.0 - 1.0
#checkpoint_warning = 30s # 0 is off
# - Archiving -

View File

@ -6,7 +6,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/access/xlog.h,v 1.78 2007/05/30 20:12:02 tgl Exp $
* $PostgreSQL: pgsql/src/include/access/xlog.h,v 1.79 2007/06/28 00:02:39 tgl Exp $
*/
#ifndef XLOG_H
#define XLOG_H
@ -157,6 +157,14 @@ extern const char XLOG_sync_method_default[];
extern bool XLOG_DEBUG;
#endif
/* OR-able flags for RequestCheckpoint, CreateCheckPoint and subsidiaries */
#define CHECKPOINT_IS_SHUTDOWN 0x0001 /* Checkpoint is for shutdown */
#define CHECKPOINT_IMMEDIATE 0x0002 /* Do it without delays */
#define CHECKPOINT_FORCE 0x0004 /* Force even if no activity */
#define CHECKPOINT_WARNONTIME 0x0008 /* Enable CheckPointWarning */
#define CHECKPOINT_WAIT 0x0010 /* Wait for completion */
extern XLogRecPtr XLogInsert(RmgrId rmid, uint8 info, XLogRecData *rdata);
extern void XLogFlush(XLogRecPtr RecPtr);
extern bool XLogNeedsFlush(XLogRecPtr RecPtr);
@ -171,9 +179,10 @@ extern void BootStrapXLOG(void);
extern void StartupXLOG(void);
extern void ShutdownXLOG(int code, Datum arg);
extern void InitXLOGAccess(void);
extern void CreateCheckPoint(bool shutdown, bool force);
extern void CreateCheckPoint(int flags);
extern void XLogPutNextOid(Oid nextOid);
extern XLogRecPtr GetRedoRecPtr(void);
extern XLogRecPtr GetInsertRecPtr(void);
extern void GetNextXidAndEpoch(TransactionId *xid, uint32 *epoch);
#endif /* XLOG_H */

View File

@ -37,7 +37,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/catalog/catversion.h,v 1.412 2007/06/15 20:56:51 tgl Exp $
* $PostgreSQL: pgsql/src/include/catalog/catversion.h,v 1.413 2007/06/28 00:02:39 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -53,6 +53,6 @@
*/
/* yyyymmddN */
#define CATALOG_VERSION_NO 200706151
#define CATALOG_VERSION_NO 200706271
#endif

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/catalog/pg_proc.h,v 1.460 2007/06/15 20:56:51 tgl Exp $
* $PostgreSQL: pgsql/src/include/catalog/pg_proc.h,v 1.461 2007/06/28 00:02:39 tgl Exp $
*
* NOTES
* The script catalog/genbki.sh reads this file and generates .bki
@ -2933,14 +2933,10 @@ DATA(insert OID = 2770 ( pg_stat_get_bgwriter_requested_checkpoints PGNSP PGUID
DESCR("Statistics: Number of backend requested checkpoints started by the bgwriter");
DATA(insert OID = 2771 ( pg_stat_get_bgwriter_buf_written_checkpoints PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_buf_written_checkpoints - _null_ ));
DESCR("Statistics: Number of buffers written by the bgwriter during checkpoints");
DATA(insert OID = 2772 ( pg_stat_get_bgwriter_buf_written_lru PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_buf_written_lru - _null_ ));
DESCR("Statistics: Number of buffers written by the bgwriter during LRU scans");
DATA(insert OID = 2773 ( pg_stat_get_bgwriter_buf_written_all PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_buf_written_all - _null_ ));
DESCR("Statistics: Number of buffers written by the bgwriter during all-buffer scans");
DATA(insert OID = 2774 ( pg_stat_get_bgwriter_maxwritten_lru PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_maxwritten_lru - _null_ ));
DESCR("Statistics: Number of times the bgwriter stopped processing when it had written too many buffers during LRU scans");
DATA(insert OID = 2775 ( pg_stat_get_bgwriter_maxwritten_all PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_maxwritten_all - _null_ ));
DESCR("Statistics: Number of times the bgwriter stopped processing when it had written too many buffers during all-buffer scans");
DATA(insert OID = 2772 ( pg_stat_get_bgwriter_buf_written_clean PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_buf_written_clean - _null_ ));
DESCR("Statistics: Number of buffers written by the bgwriter for cleaning dirty buffers");
DATA(insert OID = 2773 ( pg_stat_get_bgwriter_maxwritten_clean PGNSP PGUID 12 1 0 f f t f s 0 20 "" _null_ _null_ _null_ pg_stat_get_bgwriter_maxwritten_clean - _null_ ));
DESCR("Statistics: Number of times the bgwriter stopped processing when it had written too many buffers while cleaning");
DATA(insert OID = 2230 ( pg_stat_clear_snapshot PGNSP PGUID 12 1 0 f f f f v 0 2278 "" _null_ _null_ _null_ pg_stat_clear_snapshot - _null_ ));
DESCR("Statistics: Discard current transaction's statistics snapshot");
DATA(insert OID = 2274 ( pg_stat_reset PGNSP PGUID 12 1 0 f f f f v 0 2278 "" _null_ _null_ _null_ pg_stat_reset - _null_ ));

View File

@ -5,7 +5,7 @@
*
* Copyright (c) 2001-2007, PostgreSQL Global Development Group
*
* $PostgreSQL: pgsql/src/include/pgstat.h,v 1.62 2007/06/09 18:49:55 tgl Exp $
* $PostgreSQL: pgsql/src/include/pgstat.h,v 1.63 2007/06/28 00:02:40 tgl Exp $
* ----------
*/
#ifndef PGSTAT_H
@ -291,10 +291,8 @@ typedef struct PgStat_MsgBgWriter
PgStat_Counter m_timed_checkpoints;
PgStat_Counter m_requested_checkpoints;
PgStat_Counter m_buf_written_checkpoints;
PgStat_Counter m_buf_written_lru;
PgStat_Counter m_buf_written_all;
PgStat_Counter m_maxwritten_lru;
PgStat_Counter m_maxwritten_all;
PgStat_Counter m_buf_written_clean;
PgStat_Counter m_maxwritten_clean;
} PgStat_MsgBgWriter;
@ -392,10 +390,8 @@ typedef struct PgStat_GlobalStats
PgStat_Counter timed_checkpoints;
PgStat_Counter requested_checkpoints;
PgStat_Counter buf_written_checkpoints;
PgStat_Counter buf_written_lru;
PgStat_Counter buf_written_all;
PgStat_Counter maxwritten_lru;
PgStat_Counter maxwritten_all;
PgStat_Counter buf_written_clean;
PgStat_Counter maxwritten_clean;
} PgStat_GlobalStats;

View File

@ -5,7 +5,7 @@
*
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
*
* $PostgreSQL: pgsql/src/include/postmaster/bgwriter.h,v 1.9 2007/01/05 22:19:57 momjian Exp $
* $PostgreSQL: pgsql/src/include/postmaster/bgwriter.h,v 1.10 2007/06/28 00:02:40 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -20,10 +20,12 @@
extern int BgWriterDelay;
extern int CheckPointTimeout;
extern int CheckPointWarning;
extern double CheckPointCompletionTarget;
extern void BackgroundWriterMain(void);
extern void RequestCheckpoint(bool waitforit, bool warnontime);
extern void RequestCheckpoint(int flags);
extern void CheckpointWriteDelay(int flags, double progress);
extern bool ForwardFsyncRequest(RelFileNode rnode, BlockNumber segno);
extern void AbsorbFsyncRequests(void);

View File

@ -8,7 +8,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/storage/buf_internals.h,v 1.90 2007/05/30 20:12:03 tgl Exp $
* $PostgreSQL: pgsql/src/include/storage/buf_internals.h,v 1.91 2007/06/28 00:02:40 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -35,6 +35,7 @@
#define BM_IO_ERROR (1 << 4) /* previous I/O failed */
#define BM_JUST_DIRTIED (1 << 5) /* dirtied since write started */
#define BM_PIN_COUNT_WAITER (1 << 6) /* have waiter for sole pin */
#define BM_CHECKPOINT_NEEDED (1 << 7) /* must write for checkpoint */
typedef bits16 BufFlags;

View File

@ -7,7 +7,7 @@
* Portions Copyright (c) 1996-2007, PostgreSQL Global Development Group
* Portions Copyright (c) 1994, Regents of the University of California
*
* $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.104 2007/05/30 20:12:03 tgl Exp $
* $PostgreSQL: pgsql/src/include/storage/bufmgr.h,v 1.105 2007/06/28 00:02:40 tgl Exp $
*
*-------------------------------------------------------------------------
*/
@ -33,9 +33,7 @@ extern DLLIMPORT int NBuffers;
/* in bufmgr.c */
extern bool zero_damaged_pages;
extern double bgwriter_lru_percent;
extern double bgwriter_all_percent;
extern int bgwriter_lru_maxpages;
extern int bgwriter_all_maxpages;
/* in buf_init.c */
extern DLLIMPORT char *BufferBlocks;
@ -136,11 +134,12 @@ extern char *ShowBufferUsage(void);
extern void ResetBufferUsage(void);
extern void AtEOXact_Buffers(bool isCommit);
extern void PrintBufferLeakWarning(Buffer buffer);
extern void FlushBufferPool(void);
extern void CheckPointBuffers(int flags);
extern BlockNumber BufferGetBlockNumber(Buffer buffer);
extern BlockNumber RelationGetNumberOfBlocks(Relation relation);
extern void RelationTruncate(Relation rel, BlockNumber nblocks);
extern void FlushRelationBuffers(Relation rel);
extern void FlushDatabaseBuffers(Oid dbid);
extern void DropRelFileNodeBuffers(RelFileNode rnode, bool istemp,
BlockNumber firstDelBlock);
extern void DropDatabaseBuffers(Oid dbid);
@ -161,7 +160,6 @@ extern void LockBufferForCleanup(Buffer buffer);
extern void AbortBufferIO(void);
extern void BufmgrCommit(void);
extern void BufferSync(void);
extern void BgBufferSync(void);
extern void AtProcExit_LocalBuffers(void);

View File

@ -1292,7 +1292,7 @@ SELECT viewname, definition FROM pg_views WHERE schemaname <> 'information_schem
pg_stat_activity | SELECT d.oid AS datid, d.datname, pg_stat_get_backend_pid(s.backendid) AS procpid, pg_stat_get_backend_userid(s.backendid) AS usesysid, u.rolname AS usename, pg_stat_get_backend_activity(s.backendid) AS current_query, pg_stat_get_backend_waiting(s.backendid) AS waiting, pg_stat_get_backend_txn_start(s.backendid) AS txn_start, pg_stat_get_backend_activity_start(s.backendid) AS query_start, pg_stat_get_backend_start(s.backendid) AS backend_start, pg_stat_get_backend_client_addr(s.backendid) AS client_addr, pg_stat_get_backend_client_port(s.backendid) AS client_port FROM pg_database d, (SELECT pg_stat_get_backend_idset() AS backendid) s, pg_authid u WHERE ((pg_stat_get_backend_dbid(s.backendid) = d.oid) AND (pg_stat_get_backend_userid(s.backendid) = u.oid));
pg_stat_all_indexes | SELECT c.oid AS relid, i.oid AS indexrelid, n.nspname AS schemaname, c.relname, i.relname AS indexrelname, pg_stat_get_numscans(i.oid) AS idx_scan, pg_stat_get_tuples_returned(i.oid) AS idx_tup_read, pg_stat_get_tuples_fetched(i.oid) AS idx_tup_fetch FROM (((pg_class c JOIN pg_index x ON ((c.oid = x.indrelid))) JOIN pg_class i ON ((i.oid = x.indexrelid))) LEFT JOIN pg_namespace n ON ((n.oid = c.relnamespace))) WHERE (c.relkind = ANY (ARRAY['r'::"char", 't'::"char"]));
pg_stat_all_tables | SELECT c.oid AS relid, n.nspname AS schemaname, c.relname, pg_stat_get_numscans(c.oid) AS seq_scan, pg_stat_get_tuples_returned(c.oid) AS seq_tup_read, (sum(pg_stat_get_numscans(i.indexrelid)))::bigint AS idx_scan, ((sum(pg_stat_get_tuples_fetched(i.indexrelid)))::bigint + pg_stat_get_tuples_fetched(c.oid)) AS idx_tup_fetch, pg_stat_get_tuples_inserted(c.oid) AS n_tup_ins, pg_stat_get_tuples_updated(c.oid) AS n_tup_upd, pg_stat_get_tuples_deleted(c.oid) AS n_tup_del, pg_stat_get_live_tuples(c.oid) AS n_live_tup, pg_stat_get_dead_tuples(c.oid) AS n_dead_tup, pg_stat_get_last_vacuum_time(c.oid) AS last_vacuum, pg_stat_get_last_autovacuum_time(c.oid) AS last_autovacuum, pg_stat_get_last_analyze_time(c.oid) AS last_analyze, pg_stat_get_last_autoanalyze_time(c.oid) AS last_autoanalyze FROM ((pg_class c LEFT JOIN pg_index i ON ((c.oid = i.indrelid))) LEFT JOIN pg_namespace n ON ((n.oid = c.relnamespace))) WHERE (c.relkind = ANY (ARRAY['r'::"char", 't'::"char"])) GROUP BY c.oid, n.nspname, c.relname;
pg_stat_bgwriter | SELECT pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints_timed, pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req, pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint, pg_stat_get_bgwriter_buf_written_lru() AS buffers_lru, pg_stat_get_bgwriter_buf_written_all() AS buffers_all, pg_stat_get_bgwriter_maxwritten_lru() AS maxwritten_lru, pg_stat_get_bgwriter_maxwritten_all() AS maxwritten_all;
pg_stat_bgwriter | SELECT pg_stat_get_bgwriter_timed_checkpoints() AS checkpoints_timed, pg_stat_get_bgwriter_requested_checkpoints() AS checkpoints_req, pg_stat_get_bgwriter_buf_written_checkpoints() AS buffers_checkpoint, pg_stat_get_bgwriter_buf_written_clean() AS buffers_clean, pg_stat_get_bgwriter_maxwritten_clean() AS maxwritten_clean;
pg_stat_database | SELECT d.oid AS datid, d.datname, pg_stat_get_db_numbackends(d.oid) AS numbackends, pg_stat_get_db_xact_commit(d.oid) AS xact_commit, pg_stat_get_db_xact_rollback(d.oid) AS xact_rollback, (pg_stat_get_db_blocks_fetched(d.oid) - pg_stat_get_db_blocks_hit(d.oid)) AS blks_read, pg_stat_get_db_blocks_hit(d.oid) AS blks_hit, pg_stat_get_db_tuples_returned(d.oid) AS tup_returned, pg_stat_get_db_tuples_fetched(d.oid) AS tup_fetched, pg_stat_get_db_tuples_inserted(d.oid) AS tup_inserted, pg_stat_get_db_tuples_updated(d.oid) AS tup_updated, pg_stat_get_db_tuples_deleted(d.oid) AS tup_deleted FROM pg_database d;
pg_stat_sys_indexes | SELECT pg_stat_all_indexes.relid, pg_stat_all_indexes.indexrelid, pg_stat_all_indexes.schemaname, pg_stat_all_indexes.relname, pg_stat_all_indexes.indexrelname, pg_stat_all_indexes.idx_scan, pg_stat_all_indexes.idx_tup_read, pg_stat_all_indexes.idx_tup_fetch FROM pg_stat_all_indexes WHERE (pg_stat_all_indexes.schemaname = ANY (ARRAY['pg_catalog'::name, 'pg_toast'::name, 'information_schema'::name]));
pg_stat_sys_tables | SELECT pg_stat_all_tables.relid, pg_stat_all_tables.schemaname, pg_stat_all_tables.relname, pg_stat_all_tables.seq_scan, pg_stat_all_tables.seq_tup_read, pg_stat_all_tables.idx_scan, pg_stat_all_tables.idx_tup_fetch, pg_stat_all_tables.n_tup_ins, pg_stat_all_tables.n_tup_upd, pg_stat_all_tables.n_tup_del, pg_stat_all_tables.n_live_tup, pg_stat_all_tables.n_dead_tup, pg_stat_all_tables.last_vacuum, pg_stat_all_tables.last_autovacuum, pg_stat_all_tables.last_analyze, pg_stat_all_tables.last_autoanalyze FROM pg_stat_all_tables WHERE (pg_stat_all_tables.schemaname = ANY (ARRAY['pg_catalog'::name, 'pg_toast'::name, 'information_schema'::name]));