PostgreSQL Source Code git master
bufmgr.c File Reference
#include "postgres.h"
#include <sys/file.h>
#include <unistd.h>
#include "access/tableam.h"
#include "access/xloginsert.h"
#include "access/xlogutils.h"
#include "catalog/storage.h"
#include "catalog/storage_xlog.h"
#include "executor/instrument.h"
#include "lib/binaryheap.h"
#include "miscadmin.h"
#include "pg_trace.h"
#include "pgstat.h"
#include "postmaster/bgwriter.h"
#include "storage/aio.h"
#include "storage/buf_internals.h"
#include "storage/bufmgr.h"
#include "storage/fd.h"
#include "storage/ipc.h"
#include "storage/lmgr.h"
#include "storage/proc.h"
#include "storage/read_stream.h"
#include "storage/smgr.h"
#include "storage/standby.h"
#include "utils/memdebug.h"
#include "utils/ps_status.h"
#include "utils/rel.h"
#include "utils/resowner.h"
#include "utils/timestamp.h"
#include "lib/sort_template.h"
Include dependency graph for bufmgr.c:

Go to the source code of this file.

Data Structures

struct  PrivateRefCountEntry
 
struct  CkptTsStatus
 
struct  SMgrSortArray
 

Macros

#define BufHdrGetBlock(bufHdr)   ((Block) (BufferBlocks + ((Size) (bufHdr)->buf_id) * BLCKSZ))
 
#define BufferGetLSN(bufHdr)   (PageGetLSN(BufHdrGetBlock(bufHdr)))
 
#define LocalBufHdrGetBlock(bufHdr)    LocalBufferBlockPointers[-((bufHdr)->buf_id + 2)]
 
#define BUF_WRITTEN   0x01
 
#define BUF_REUSABLE   0x02
 
#define RELS_BSEARCH_THRESHOLD   20
 
#define BUF_DROP_FULL_SCAN_THRESHOLD   (uint64) (NBuffers / 32)
 
#define REFCOUNT_ARRAY_ENTRIES   8
 
#define BufferIsPinned(bufnum)
 
#define ST_SORT   sort_checkpoint_bufferids
 
#define ST_ELEMENT_TYPE   CkptSortItem
 
#define ST_COMPARE(a, b)   ckpt_buforder_comparator(a, b)
 
#define ST_SCOPE   static
 
#define ST_DEFINE
 
#define ST_SORT   sort_pending_writebacks
 
#define ST_ELEMENT_TYPE   PendingWriteback
 
#define ST_COMPARE(a, b)   buffertag_comparator(&a->tag, &b->tag)
 
#define ST_SCOPE   static
 
#define ST_DEFINE
 
#define READV_COUNT_BITS   7
 
#define READV_COUNT_MASK   ((1 << READV_COUNT_BITS) - 1)
 

Typedefs

typedef struct PrivateRefCountEntry PrivateRefCountEntry
 
typedef struct CkptTsStatus CkptTsStatus
 
typedef struct SMgrSortArray SMgrSortArray
 

Functions

static void ReservePrivateRefCountEntry (void)
 
static PrivateRefCountEntryNewPrivateRefCountEntry (Buffer buffer)
 
static PrivateRefCountEntryGetPrivateRefCountEntry (Buffer buffer, bool do_move)
 
static int32 GetPrivateRefCount (Buffer buffer)
 
static void ForgetPrivateRefCountEntry (PrivateRefCountEntry *ref)
 
static void ResOwnerReleaseBufferIO (Datum res)
 
static char * ResOwnerPrintBufferIO (Datum res)
 
static void ResOwnerReleaseBufferPin (Datum res)
 
static char * ResOwnerPrintBufferPin (Datum res)
 
static Buffer ReadBuffer_common (Relation rel, SMgrRelation smgr, char smgr_persistence, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy)
 
static BlockNumber ExtendBufferedRelCommon (BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, BlockNumber extend_upto, Buffer *buffers, uint32 *extended_by)
 
static BlockNumber ExtendBufferedRelShared (BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, BlockNumber extend_upto, Buffer *buffers, uint32 *extended_by)
 
static bool PinBuffer (BufferDesc *buf, BufferAccessStrategy strategy)
 
static void PinBuffer_Locked (BufferDesc *buf)
 
static void UnpinBuffer (BufferDesc *buf)
 
static void UnpinBufferNoOwner (BufferDesc *buf)
 
static void BufferSync (int flags)
 
static uint32 WaitBufHdrUnlocked (BufferDesc *buf)
 
static int SyncOneBuffer (int buf_id, bool skip_recently_used, WritebackContext *wb_context)
 
static void WaitIO (BufferDesc *buf)
 
static void AbortBufferIO (Buffer buffer)
 
static void shared_buffer_write_error_callback (void *arg)
 
static void local_buffer_write_error_callback (void *arg)
 
static BufferDescBufferAlloc (SMgrRelation smgr, char relpersistence, ForkNumber forkNum, BlockNumber blockNum, BufferAccessStrategy strategy, bool *foundPtr, IOContext io_context)
 
static bool AsyncReadBuffers (ReadBuffersOperation *operation, int *nblocks_progress)
 
static void CheckReadBuffersOperation (ReadBuffersOperation *operation, bool is_complete)
 
static Buffer GetVictimBuffer (BufferAccessStrategy strategy, IOContext io_context)
 
static void FlushBuffer (BufferDesc *buf, SMgrRelation reln, IOObject io_object, IOContext io_context)
 
static void FindAndDropRelationBuffers (RelFileLocator rlocator, ForkNumber forkNum, BlockNumber nForkBlock, BlockNumber firstDelBlock)
 
static void RelationCopyStorageUsingBuffer (RelFileLocator srclocator, RelFileLocator dstlocator, ForkNumber forkNum, bool permanent)
 
static void AtProcExit_Buffers (int code, Datum arg)
 
static void CheckForBufferLeaks (void)
 
static int rlocator_comparator (const void *p1, const void *p2)
 
static int buffertag_comparator (const BufferTag *ba, const BufferTag *bb)
 
static int ckpt_buforder_comparator (const CkptSortItem *a, const CkptSortItem *b)
 
static int ts_ckpt_progress_comparator (Datum a, Datum b, void *arg)
 
PrefetchBufferResult PrefetchSharedBuffer (SMgrRelation smgr_reln, ForkNumber forkNum, BlockNumber blockNum)
 
PrefetchBufferResult PrefetchBuffer (Relation reln, ForkNumber forkNum, BlockNumber blockNum)
 
bool ReadRecentBuffer (RelFileLocator rlocator, ForkNumber forkNum, BlockNumber blockNum, Buffer recent_buffer)
 
Buffer ReadBuffer (Relation reln, BlockNumber blockNum)
 
Buffer ReadBufferExtended (Relation reln, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy)
 
Buffer ReadBufferWithoutRelcache (RelFileLocator rlocator, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy, bool permanent)
 
Buffer ExtendBufferedRel (BufferManagerRelation bmr, ForkNumber forkNum, BufferAccessStrategy strategy, uint32 flags)
 
BlockNumber ExtendBufferedRelBy (BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, Buffer *buffers, uint32 *extended_by)
 
Buffer ExtendBufferedRelTo (BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, BlockNumber extend_to, ReadBufferMode mode)
 
static void ZeroAndLockBuffer (Buffer buffer, ReadBufferMode mode, bool already_valid)
 
static pg_attribute_always_inline Buffer PinBufferForBlock (Relation rel, SMgrRelation smgr, char persistence, ForkNumber forkNum, BlockNumber blockNum, BufferAccessStrategy strategy, bool *foundPtr)
 
static pg_attribute_always_inline bool StartReadBuffersImpl (ReadBuffersOperation *operation, Buffer *buffers, BlockNumber blockNum, int *nblocks, int flags, bool allow_forwarding)
 
bool StartReadBuffers (ReadBuffersOperation *operation, Buffer *buffers, BlockNumber blockNum, int *nblocks, int flags)
 
bool StartReadBuffer (ReadBuffersOperation *operation, Buffer *buffer, BlockNumber blocknum, int flags)
 
static bool ReadBuffersCanStartIOOnce (Buffer buffer, bool nowait)
 
static bool ReadBuffersCanStartIO (Buffer buffer, bool nowait)
 
static void ProcessReadBuffersResult (ReadBuffersOperation *operation)
 
void WaitReadBuffers (ReadBuffersOperation *operation)
 
static void InvalidateBuffer (BufferDesc *buf)
 
static bool InvalidateVictimBuffer (BufferDesc *buf_hdr)
 
uint32 GetPinLimit (void)
 
uint32 GetAdditionalPinLimit (void)
 
void LimitAdditionalPins (uint32 *additional_pins)
 
bool BufferIsExclusiveLocked (Buffer buffer)
 
bool BufferIsDirty (Buffer buffer)
 
void MarkBufferDirty (Buffer buffer)
 
Buffer ReleaseAndReadBuffer (Buffer buffer, Relation relation, BlockNumber blockNum)
 
static void WakePinCountWaiter (BufferDesc *buf)
 
bool BgBufferSync (WritebackContext *wb_context)
 
void AtEOXact_Buffers (bool isCommit)
 
void InitBufferManagerAccess (void)
 
char * DebugPrintBufferRefcount (Buffer buffer)
 
void CheckPointBuffers (int flags)
 
BlockNumber BufferGetBlockNumber (Buffer buffer)
 
void BufferGetTag (Buffer buffer, RelFileLocator *rlocator, ForkNumber *forknum, BlockNumber *blknum)
 
BlockNumber RelationGetNumberOfBlocksInFork (Relation relation, ForkNumber forkNum)
 
bool BufferIsPermanent (Buffer buffer)
 
XLogRecPtr BufferGetLSNAtomic (Buffer buffer)
 
void DropRelationBuffers (SMgrRelation smgr_reln, ForkNumber *forkNum, int nforks, BlockNumber *firstDelBlock)
 
void DropRelationsAllBuffers (SMgrRelation *smgr_reln, int nlocators)
 
void DropDatabaseBuffers (Oid dbid)
 
void FlushRelationBuffers (Relation rel)
 
void FlushRelationsAllBuffers (SMgrRelation *smgrs, int nrels)
 
void CreateAndCopyRelationData (RelFileLocator src_rlocator, RelFileLocator dst_rlocator, bool permanent)
 
void FlushDatabaseBuffers (Oid dbid)
 
void FlushOneBuffer (Buffer buffer)
 
void ReleaseBuffer (Buffer buffer)
 
void UnlockReleaseBuffer (Buffer buffer)
 
void IncrBufferRefCount (Buffer buffer)
 
void MarkBufferDirtyHint (Buffer buffer, bool buffer_std)
 
void UnlockBuffers (void)
 
void LockBuffer (Buffer buffer, int mode)
 
bool ConditionalLockBuffer (Buffer buffer)
 
void CheckBufferIsPinnedOnce (Buffer buffer)
 
void LockBufferForCleanup (Buffer buffer)
 
bool HoldingBufferPinThatDelaysRecovery (void)
 
bool ConditionalLockBufferForCleanup (Buffer buffer)
 
bool IsBufferCleanupOK (Buffer buffer)
 
bool StartBufferIO (BufferDesc *buf, bool forInput, bool nowait)
 
void TerminateBufferIO (BufferDesc *buf, bool clear_dirty, uint32 set_flag_bits, bool forget_owner, bool release_aio)
 
uint32 LockBufHdr (BufferDesc *desc)
 
void WritebackContextInit (WritebackContext *context, int *max_pending)
 
void ScheduleBufferTagForWriteback (WritebackContext *wb_context, IOContext io_context, BufferTag *tag)
 
void IssuePendingWritebacks (WritebackContext *wb_context, IOContext io_context)
 
static bool EvictUnpinnedBufferInternal (BufferDesc *desc, bool *buffer_flushed)
 
bool EvictUnpinnedBuffer (Buffer buf, bool *buffer_flushed)
 
void EvictAllUnpinnedBuffers (int32 *buffers_evicted, int32 *buffers_flushed, int32 *buffers_skipped)
 
void EvictRelUnpinnedBuffers (Relation rel, int32 *buffers_evicted, int32 *buffers_flushed, int32 *buffers_skipped)
 
static pg_attribute_always_inline void buffer_stage_common (PgAioHandle *ioh, bool is_write, bool is_temp)
 
static void buffer_readv_decode_error (PgAioResult result, bool *zeroed_any, bool *ignored_any, uint8 *zeroed_or_error_count, uint8 *checkfail_count, uint8 *first_off)
 
static void buffer_readv_encode_error (PgAioResult *result, bool is_temp, bool zeroed_any, bool ignored_any, uint8 error_count, uint8 zeroed_count, uint8 checkfail_count, uint8 first_error_off, uint8 first_zeroed_off, uint8 first_ignored_off)
 
static pg_attribute_always_inline void buffer_readv_complete_one (PgAioTargetData *td, uint8 buf_off, Buffer buffer, uint8 flags, bool failed, bool is_temp, bool *buffer_invalid, bool *failed_checksum, bool *ignored_checksum, bool *zeroed_buffer)
 
static pg_attribute_always_inline PgAioResult buffer_readv_complete (PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data, bool is_temp)
 
static void buffer_readv_report (PgAioResult result, const PgAioTargetData *td, int elevel)
 
static void shared_buffer_readv_stage (PgAioHandle *ioh, uint8 cb_data)
 
static PgAioResult shared_buffer_readv_complete (PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
 
static PgAioResult shared_buffer_readv_complete_local (PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
 
static void local_buffer_readv_stage (PgAioHandle *ioh, uint8 cb_data)
 
static PgAioResult local_buffer_readv_complete (PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
 

Variables

bool zero_damaged_pages = false
 
int bgwriter_lru_maxpages = 100
 
double bgwriter_lru_multiplier = 2.0
 
bool track_io_timing = false
 
int effective_io_concurrency = DEFAULT_EFFECTIVE_IO_CONCURRENCY
 
int maintenance_io_concurrency = DEFAULT_MAINTENANCE_IO_CONCURRENCY
 
int io_combine_limit = DEFAULT_IO_COMBINE_LIMIT
 
int io_combine_limit_guc = DEFAULT_IO_COMBINE_LIMIT
 
int io_max_combine_limit = DEFAULT_IO_COMBINE_LIMIT
 
int checkpoint_flush_after = DEFAULT_CHECKPOINT_FLUSH_AFTER
 
int bgwriter_flush_after = DEFAULT_BGWRITER_FLUSH_AFTER
 
int backend_flush_after = DEFAULT_BACKEND_FLUSH_AFTER
 
static BufferDescPinCountWaitBuf = NULL
 
static struct PrivateRefCountEntry PrivateRefCountArray [REFCOUNT_ARRAY_ENTRIES]
 
static HTABPrivateRefCountHash = NULL
 
static int32 PrivateRefCountOverflowed = 0
 
static uint32 PrivateRefCountClock = 0
 
static PrivateRefCountEntryReservedRefCountEntry = NULL
 
static uint32 MaxProportionalPins
 
const ResourceOwnerDesc buffer_io_resowner_desc
 
const ResourceOwnerDesc buffer_pin_resowner_desc
 
const PgAioHandleCallbacks aio_shared_buffer_readv_cb
 
const PgAioHandleCallbacks aio_local_buffer_readv_cb
 

Macro Definition Documentation

◆ BUF_DROP_FULL_SCAN_THRESHOLD

#define BUF_DROP_FULL_SCAN_THRESHOLD   (uint64) (NBuffers / 32)

Definition at line 91 of file bufmgr.c.

◆ BUF_REUSABLE

#define BUF_REUSABLE   0x02

Definition at line 81 of file bufmgr.c.

◆ BUF_WRITTEN

#define BUF_WRITTEN   0x01

Definition at line 80 of file bufmgr.c.

◆ BufferGetLSN

#define BufferGetLSN (   bufHdr)    (PageGetLSN(BufHdrGetBlock(bufHdr)))

Definition at line 73 of file bufmgr.c.

◆ BufferIsPinned

#define BufferIsPinned (   bufnum)
Value:
( \
!BufferIsValid(bufnum) ? \
false \
: \
BufferIsLocal(bufnum) ? \
(LocalRefCount[-(bufnum) - 1] > 0) \
: \
(GetPrivateRefCount(bufnum) > 0) \
)
static int32 GetPrivateRefCount(Buffer buffer)
Definition: bufmgr.c:425
static bool BufferIsValid(Buffer bufnum)
Definition: bufmgr.h:368
int32 * LocalRefCount
Definition: localbuf.c:48

Definition at line 483 of file bufmgr.c.

◆ BufHdrGetBlock

#define BufHdrGetBlock (   bufHdr)    ((Block) (BufferBlocks + ((Size) (bufHdr)->buf_id) * BLCKSZ))

Definition at line 72 of file bufmgr.c.

◆ LocalBufHdrGetBlock

#define LocalBufHdrGetBlock (   bufHdr)     LocalBufferBlockPointers[-((bufHdr)->buf_id + 2)]

Definition at line 76 of file bufmgr.c.

◆ READV_COUNT_BITS

#define READV_COUNT_BITS   7

◆ READV_COUNT_MASK

#define READV_COUNT_MASK   ((1 << READV_COUNT_BITS) - 1)

◆ REFCOUNT_ARRAY_ENTRIES

#define REFCOUNT_ARRAY_ENTRIES   8

Definition at line 100 of file bufmgr.c.

◆ RELS_BSEARCH_THRESHOLD

#define RELS_BSEARCH_THRESHOLD   20

Definition at line 83 of file bufmgr.c.

◆ ST_COMPARE [1/2]

#define ST_COMPARE (   a,
  b 
)    ckpt_buforder_comparator(a, b)

Definition at line 6448 of file bufmgr.c.

◆ ST_COMPARE [2/2]

#define ST_COMPARE (   a,
  b 
)    buffertag_comparator(&a->tag, &b->tag)

Definition at line 6448 of file bufmgr.c.

◆ ST_DEFINE [1/2]

#define ST_DEFINE

Definition at line 6450 of file bufmgr.c.

◆ ST_DEFINE [2/2]

#define ST_DEFINE

Definition at line 6450 of file bufmgr.c.

◆ ST_ELEMENT_TYPE [1/2]

#define ST_ELEMENT_TYPE   CkptSortItem

Definition at line 6447 of file bufmgr.c.

◆ ST_ELEMENT_TYPE [2/2]

#define ST_ELEMENT_TYPE   PendingWriteback

Definition at line 6447 of file bufmgr.c.

◆ ST_SCOPE [1/2]

#define ST_SCOPE   static

Definition at line 6449 of file bufmgr.c.

◆ ST_SCOPE [2/2]

#define ST_SCOPE   static

Definition at line 6449 of file bufmgr.c.

◆ ST_SORT [1/2]

#define ST_SORT   sort_checkpoint_bufferids

Definition at line 6446 of file bufmgr.c.

◆ ST_SORT [2/2]

#define ST_SORT   sort_pending_writebacks

Definition at line 6446 of file bufmgr.c.

Typedef Documentation

◆ CkptTsStatus

typedef struct CkptTsStatus CkptTsStatus

◆ PrivateRefCountEntry

◆ SMgrSortArray

typedef struct SMgrSortArray SMgrSortArray

Function Documentation

◆ AbortBufferIO()

static void AbortBufferIO ( Buffer  buffer)
static

Definition at line 6159 of file bufmgr.c.

6160{
6161 BufferDesc *buf_hdr = GetBufferDescriptor(buffer - 1);
6162 uint32 buf_state;
6163
6164 buf_state = LockBufHdr(buf_hdr);
6165 Assert(buf_state & (BM_IO_IN_PROGRESS | BM_TAG_VALID));
6166
6167 if (!(buf_state & BM_VALID))
6168 {
6169 Assert(!(buf_state & BM_DIRTY));
6170 UnlockBufHdr(buf_hdr, buf_state);
6171 }
6172 else
6173 {
6174 Assert(buf_state & BM_DIRTY);
6175 UnlockBufHdr(buf_hdr, buf_state);
6176
6177 /* Issue notice if this is not the first failure... */
6178 if (buf_state & BM_IO_ERROR)
6179 {
6180 /* Buffer is pinned, so we can read tag without spinlock */
6182 (errcode(ERRCODE_IO_ERROR),
6183 errmsg("could not write block %u of %s",
6184 buf_hdr->tag.blockNum,
6186 BufTagGetForkNum(&buf_hdr->tag)).str),
6187 errdetail("Multiple failures --- write error might be permanent.")));
6188 }
6189 }
6190
6191 TerminateBufferIO(buf_hdr, false, BM_IO_ERROR, false, false);
6192}
#define BM_TAG_VALID
Definition: buf_internals.h:71
static ForkNumber BufTagGetForkNum(const BufferTag *tag)
static void UnlockBufHdr(BufferDesc *desc, uint32 buf_state)
#define BM_DIRTY
Definition: buf_internals.h:69
#define BM_IO_IN_PROGRESS
Definition: buf_internals.h:72
static RelFileLocator BufTagGetRelFileLocator(const BufferTag *tag)
#define BM_VALID
Definition: buf_internals.h:70
#define BM_IO_ERROR
Definition: buf_internals.h:73
static BufferDesc * GetBufferDescriptor(uint32 id)
void TerminateBufferIO(BufferDesc *buf, bool clear_dirty, uint32 set_flag_bits, bool forget_owner, bool release_aio)
Definition: bufmgr.c:6100
uint32 LockBufHdr(BufferDesc *desc)
Definition: bufmgr.c:6257
uint32_t uint32
Definition: c.h:502
int errdetail(const char *fmt,...)
Definition: elog.c:1204
int errcode(int sqlerrcode)
Definition: elog.c:854
int errmsg(const char *fmt,...)
Definition: elog.c:1071
#define WARNING
Definition: elog.h:36
#define ereport(elevel,...)
Definition: elog.h:149
Assert(PointerIsAligned(start, uint64))
#define relpathperm(rlocator, forknum)
Definition: relpath.h:146
BufferTag tag
BlockNumber blockNum

References Assert(), buftag::blockNum, BM_DIRTY, BM_IO_ERROR, BM_IO_IN_PROGRESS, BM_TAG_VALID, BM_VALID, PrivateRefCountEntry::buffer, BufTagGetForkNum(), BufTagGetRelFileLocator(), ereport, errcode(), errdetail(), errmsg(), GetBufferDescriptor(), LockBufHdr(), relpathperm, BufferDesc::tag, TerminateBufferIO(), UnlockBufHdr(), and WARNING.

Referenced by ResOwnerReleaseBufferIO().

◆ AsyncReadBuffers()

static bool AsyncReadBuffers ( ReadBuffersOperation operation,
int *  nblocks_progress 
)
static

Definition at line 1769 of file bufmgr.c.

1770{
1771 Buffer *buffers = &operation->buffers[0];
1772 int flags = operation->flags;
1773 BlockNumber blocknum = operation->blocknum;
1774 ForkNumber forknum = operation->forknum;
1775 char persistence = operation->persistence;
1776 int16 nblocks_done = operation->nblocks_done;
1777 Buffer *io_buffers = &operation->buffers[nblocks_done];
1778 int io_buffers_len = 0;
1779 PgAioHandle *ioh;
1780 uint32 ioh_flags = 0;
1781 void *io_pages[MAX_IO_COMBINE_LIMIT];
1782 IOContext io_context;
1783 IOObject io_object;
1784 bool did_start_io;
1785
1786 /*
1787 * When this IO is executed synchronously, either because the caller will
1788 * immediately block waiting for the IO or because IOMETHOD_SYNC is used,
1789 * the AIO subsystem needs to know.
1790 */
1791 if (flags & READ_BUFFERS_SYNCHRONOUSLY)
1792 ioh_flags |= PGAIO_HF_SYNCHRONOUS;
1793
1794 if (persistence == RELPERSISTENCE_TEMP)
1795 {
1796 io_context = IOCONTEXT_NORMAL;
1797 io_object = IOOBJECT_TEMP_RELATION;
1798 ioh_flags |= PGAIO_HF_REFERENCES_LOCAL;
1799 }
1800 else
1801 {
1802 io_context = IOContextForStrategy(operation->strategy);
1803 io_object = IOOBJECT_RELATION;
1804 }
1805
1806 /*
1807 * If zero_damaged_pages is enabled, add the READ_BUFFERS_ZERO_ON_ERROR
1808 * flag. The reason for that is that, hopefully, zero_damaged_pages isn't
1809 * set globally, but on a per-session basis. The completion callback,
1810 * which may be run in other processes, e.g. in IO workers, may have a
1811 * different value of the zero_damaged_pages GUC.
1812 *
1813 * XXX: We probably should eventually use a different flag for
1814 * zero_damaged_pages, so we can report different log levels / error codes
1815 * for zero_damaged_pages and ZERO_ON_ERROR.
1816 */
1819
1820 /*
1821 * For the same reason as with zero_damaged_pages we need to use this
1822 * backend's ignore_checksum_failure value.
1823 */
1826
1827
1828 /*
1829 * To be allowed to report stats in the local completion callback we need
1830 * to prepare to report stats now. This ensures we can safely report the
1831 * checksum failure even in a critical section.
1832 */
1834
1835 /*
1836 * Get IO handle before ReadBuffersCanStartIO(), as pgaio_io_acquire()
1837 * might block, which we don't want after setting IO_IN_PROGRESS.
1838 *
1839 * If we need to wait for IO before we can get a handle, submit
1840 * already-staged IO first, so that other backends don't need to wait.
1841 * There wouldn't be a deadlock risk, as pgaio_io_acquire() just needs to
1842 * wait for already submitted IO, which doesn't require additional locks,
1843 * but it could still cause undesirable waits.
1844 *
1845 * A secondary benefit is that this would allow us to measure the time in
1846 * pgaio_io_acquire() without causing undue timer overhead in the common,
1847 * non-blocking, case. However, currently the pgstats infrastructure
1848 * doesn't really allow that, as it a) asserts that an operation can't
1849 * have time without operations b) doesn't have an API to report
1850 * "accumulated" time.
1851 */
1853 if (unlikely(!ioh))
1854 {
1856
1858 }
1859
1860 /*
1861 * Check if we can start IO on the first to-be-read buffer.
1862 *
1863 * If an I/O is already in progress in another backend, we want to wait
1864 * for the outcome: either done, or something went wrong and we will
1865 * retry.
1866 */
1867 if (!ReadBuffersCanStartIO(buffers[nblocks_done], false))
1868 {
1869 /*
1870 * Someone else has already completed this block, we're done.
1871 *
1872 * When IO is necessary, ->nblocks_done is updated in
1873 * ProcessReadBuffersResult(), but that is not called if no IO is
1874 * necessary. Thus update here.
1875 */
1876 operation->nblocks_done += 1;
1877 *nblocks_progress = 1;
1878
1879 pgaio_io_release(ioh);
1880 pgaio_wref_clear(&operation->io_wref);
1881 did_start_io = false;
1882
1883 /*
1884 * Report and track this as a 'hit' for this backend, even though it
1885 * must have started out as a miss in PinBufferForBlock(). The other
1886 * backend will track this as a 'read'.
1887 */
1888 TRACE_POSTGRESQL_BUFFER_READ_DONE(forknum, blocknum + operation->nblocks_done,
1889 operation->smgr->smgr_rlocator.locator.spcOid,
1890 operation->smgr->smgr_rlocator.locator.dbOid,
1891 operation->smgr->smgr_rlocator.locator.relNumber,
1892 operation->smgr->smgr_rlocator.backend,
1893 true);
1894
1895 if (persistence == RELPERSISTENCE_TEMP)
1897 else
1899
1900 if (operation->rel)
1901 pgstat_count_buffer_hit(operation->rel);
1902
1903 pgstat_count_io_op(io_object, io_context, IOOP_HIT, 1, 0);
1904
1905 if (VacuumCostActive)
1907 }
1908 else
1909 {
1910 instr_time io_start;
1911
1912 /* We found a buffer that we need to read in. */
1913 Assert(io_buffers[0] == buffers[nblocks_done]);
1914 io_pages[0] = BufferGetBlock(buffers[nblocks_done]);
1915 io_buffers_len = 1;
1916
1917 /*
1918 * How many neighboring-on-disk blocks can we scatter-read into other
1919 * buffers at the same time? In this case we don't wait if we see an
1920 * I/O already in progress. We already set BM_IO_IN_PROGRESS for the
1921 * head block, so we should get on with that I/O as soon as possible.
1922 */
1923 for (int i = nblocks_done + 1; i < operation->nblocks; i++)
1924 {
1925 if (!ReadBuffersCanStartIO(buffers[i], true))
1926 break;
1927 /* Must be consecutive block numbers. */
1928 Assert(BufferGetBlockNumber(buffers[i - 1]) ==
1929 BufferGetBlockNumber(buffers[i]) - 1);
1930 Assert(io_buffers[io_buffers_len] == buffers[i]);
1931
1932 io_pages[io_buffers_len++] = BufferGetBlock(buffers[i]);
1933 }
1934
1935 /* get a reference to wait for in WaitReadBuffers() */
1936 pgaio_io_get_wref(ioh, &operation->io_wref);
1937
1938 /* provide the list of buffers to the completion callbacks */
1939 pgaio_io_set_handle_data_32(ioh, (uint32 *) io_buffers, io_buffers_len);
1940
1942 persistence == RELPERSISTENCE_TEMP ?
1945 flags);
1946
1947 pgaio_io_set_flag(ioh, ioh_flags);
1948
1949 /* ---
1950 * Even though we're trying to issue IO asynchronously, track the time
1951 * in smgrstartreadv():
1952 * - if io_method == IOMETHOD_SYNC, we will always perform the IO
1953 * immediately
1954 * - the io method might not support the IO (e.g. worker IO for a temp
1955 * table)
1956 * ---
1957 */
1959 smgrstartreadv(ioh, operation->smgr, forknum,
1960 blocknum + nblocks_done,
1961 io_pages, io_buffers_len);
1962 pgstat_count_io_op_time(io_object, io_context, IOOP_READ,
1963 io_start, 1, io_buffers_len * BLCKSZ);
1964
1965 if (persistence == RELPERSISTENCE_TEMP)
1966 pgBufferUsage.local_blks_read += io_buffers_len;
1967 else
1968 pgBufferUsage.shared_blks_read += io_buffers_len;
1969
1970 /*
1971 * Track vacuum cost when issuing IO, not after waiting for it.
1972 * Otherwise we could end up issuing a lot of IO in a short timespan,
1973 * despite a low cost limit.
1974 */
1975 if (VacuumCostActive)
1976 VacuumCostBalance += VacuumCostPageMiss * io_buffers_len;
1977
1978 *nblocks_progress = io_buffers_len;
1979 did_start_io = true;
1980 }
1981
1982 return did_start_io;
1983}
PgAioHandle * pgaio_io_acquire(struct ResourceOwnerData *resowner, PgAioReturn *ret)
Definition: aio.c:159
void pgaio_wref_clear(PgAioWaitRef *iow)
Definition: aio.c:944
void pgaio_io_get_wref(PgAioHandle *ioh, PgAioWaitRef *iow)
Definition: aio.c:363
void pgaio_io_set_flag(PgAioHandle *ioh, PgAioHandleFlags flag)
Definition: aio.c:327
void pgaio_submit_staged(void)
Definition: aio.c:1103
void pgaio_io_release(PgAioHandle *ioh)
Definition: aio.c:237
PgAioHandle * pgaio_io_acquire_nb(struct ResourceOwnerData *resowner, PgAioReturn *ret)
Definition: aio.c:185
@ PGAIO_HCB_LOCAL_BUFFER_READV
Definition: aio.h:200
@ PGAIO_HCB_SHARED_BUFFER_READV
Definition: aio.h:198
@ PGAIO_HF_SYNCHRONOUS
Definition: aio.h:70
@ PGAIO_HF_REFERENCES_LOCAL
Definition: aio.h:60
void pgaio_io_set_handle_data_32(PgAioHandle *ioh, uint32 *data, uint8 len)
Definition: aio_callback.c:140
void pgaio_io_register_callbacks(PgAioHandle *ioh, PgAioHandleCallbackID cb_id, uint8 cb_data)
Definition: aio_callback.c:86
uint32 BlockNumber
Definition: block.h:31
int Buffer
Definition: buf.h:23
bool track_io_timing
Definition: bufmgr.c:147
BlockNumber BufferGetBlockNumber(Buffer buffer)
Definition: bufmgr.c:4229
static bool ReadBuffersCanStartIO(Buffer buffer, bool nowait)
Definition: bufmgr.c:1569
bool zero_damaged_pages
Definition: bufmgr.c:144
#define READ_BUFFERS_ZERO_ON_ERROR
Definition: bufmgr.h:112
static Block BufferGetBlock(Buffer buffer)
Definition: bufmgr.h:384
#define MAX_IO_COMBINE_LIMIT
Definition: bufmgr.h:166
#define READ_BUFFERS_IGNORE_CHECKSUM_FAILURES
Definition: bufmgr.h:116
#define READ_BUFFERS_SYNCHRONOUSLY
Definition: bufmgr.h:118
bool ignore_checksum_failure
Definition: bufpage.c:27
int16_t int16
Definition: c.h:497
#define unlikely(x)
Definition: c.h:347
IOContext IOContextForStrategy(BufferAccessStrategy strategy)
Definition: freelist.c:800
int VacuumCostPageMiss
Definition: globals.c:153
bool VacuumCostActive
Definition: globals.c:159
int VacuumCostBalance
Definition: globals.c:158
int VacuumCostPageHit
Definition: globals.c:152
BufferUsage pgBufferUsage
Definition: instrument.c:20
int i
Definition: isn.c:77
IOObject
Definition: pgstat.h:273
@ IOOBJECT_RELATION
Definition: pgstat.h:274
@ IOOBJECT_TEMP_RELATION
Definition: pgstat.h:275
IOContext
Definition: pgstat.h:282
@ IOCONTEXT_NORMAL
Definition: pgstat.h:286
@ IOOP_READ
Definition: pgstat.h:312
@ IOOP_HIT
Definition: pgstat.h:306
#define pgstat_count_buffer_hit(rel)
Definition: pgstat.h:709
void pgstat_prepare_report_checksum_failure(Oid dboid)
instr_time pgstat_prepare_io_time(bool track_io_guc)
Definition: pgstat_io.c:90
void pgstat_count_io_op(IOObject io_object, IOContext io_context, IOOp io_op, uint32 cnt, uint64 bytes)
Definition: pgstat_io.c:68
void pgstat_count_io_op_time(IOObject io_object, IOContext io_context, IOOp io_op, instr_time start_time, uint32 cnt, uint64 bytes)
Definition: pgstat_io.c:121
ForkNumber
Definition: relpath.h:56
ResourceOwner CurrentResourceOwner
Definition: resowner.c:173
void smgrstartreadv(PgAioHandle *ioh, SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, void **buffers, BlockNumber nblocks)
Definition: smgr.c:753
int64 local_blks_hit
Definition: instrument.h:30
int64 shared_blks_read
Definition: instrument.h:27
int64 local_blks_read
Definition: instrument.h:31
int64 shared_blks_hit
Definition: instrument.h:26
ForkNumber forknum
Definition: bufmgr.h:127
PgAioWaitRef io_wref
Definition: bufmgr.h:140
Buffer * buffers
Definition: bufmgr.h:135
BufferAccessStrategy strategy
Definition: bufmgr.h:128
BlockNumber blocknum
Definition: bufmgr.h:136
PgAioReturn io_return
Definition: bufmgr.h:141
struct SMgrRelationData * smgr
Definition: bufmgr.h:125
RelFileLocator locator
RelFileNumber relNumber
RelFileLocatorBackend smgr_rlocator
Definition: smgr.h:38

References Assert(), RelFileLocatorBackend::backend, ReadBuffersOperation::blocknum, BufferGetBlock(), BufferGetBlockNumber(), ReadBuffersOperation::buffers, CurrentResourceOwner, RelFileLocator::dbOid, ReadBuffersOperation::flags, ReadBuffersOperation::forknum, i, ignore_checksum_failure, ReadBuffersOperation::io_return, ReadBuffersOperation::io_wref, IOCONTEXT_NORMAL, IOContextForStrategy(), IOOBJECT_RELATION, IOOBJECT_TEMP_RELATION, IOOP_HIT, IOOP_READ, BufferUsage::local_blks_hit, BufferUsage::local_blks_read, RelFileLocatorBackend::locator, MAX_IO_COMBINE_LIMIT, ReadBuffersOperation::nblocks, ReadBuffersOperation::nblocks_done, ReadBuffersOperation::persistence, PGAIO_HCB_LOCAL_BUFFER_READV, PGAIO_HCB_SHARED_BUFFER_READV, PGAIO_HF_REFERENCES_LOCAL, PGAIO_HF_SYNCHRONOUS, pgaio_io_acquire(), pgaio_io_acquire_nb(), pgaio_io_get_wref(), pgaio_io_register_callbacks(), pgaio_io_release(), pgaio_io_set_flag(), pgaio_io_set_handle_data_32(), pgaio_submit_staged(), pgaio_wref_clear(), pgBufferUsage, pgstat_count_buffer_hit, pgstat_count_io_op(), pgstat_count_io_op_time(), pgstat_prepare_io_time(), pgstat_prepare_report_checksum_failure(), READ_BUFFERS_IGNORE_CHECKSUM_FAILURES, READ_BUFFERS_SYNCHRONOUSLY, READ_BUFFERS_ZERO_ON_ERROR, ReadBuffersCanStartIO(), ReadBuffersOperation::rel, RelFileLocator::relNumber, BufferUsage::shared_blks_hit, BufferUsage::shared_blks_read, ReadBuffersOperation::smgr, SMgrRelationData::smgr_rlocator, smgrstartreadv(), RelFileLocator::spcOid, ReadBuffersOperation::strategy, track_io_timing, unlikely, VacuumCostActive, VacuumCostBalance, VacuumCostPageHit, VacuumCostPageMiss, and zero_damaged_pages.

Referenced by StartReadBuffersImpl(), and WaitReadBuffers().

◆ AtEOXact_Buffers()

void AtEOXact_Buffers ( bool  isCommit)

Definition at line 3996 of file bufmgr.c.

3997{
3999
4000 AtEOXact_LocalBuffers(isCommit);
4001
4003}
static void CheckForBufferLeaks(void)
Definition: bufmgr.c:4065
static int32 PrivateRefCountOverflowed
Definition: bufmgr.c:217
void AtEOXact_LocalBuffers(bool isCommit)
Definition: localbuf.c:993

References Assert(), AtEOXact_LocalBuffers(), CheckForBufferLeaks(), and PrivateRefCountOverflowed.

Referenced by AbortTransaction(), AutoVacLauncherMain(), BackgroundWriterMain(), CheckpointerMain(), CommitTransaction(), PrepareTransaction(), and WalWriterMain().

◆ AtProcExit_Buffers()

static void AtProcExit_Buffers ( int  code,
Datum  arg 
)
static

Definition at line 4047 of file bufmgr.c.

4048{
4049 UnlockBuffers();
4050
4052
4053 /* localbuf.c needs a chance too */
4055}
void UnlockBuffers(void)
Definition: bufmgr.c:5577
void AtProcExit_LocalBuffers(void)
Definition: localbuf.c:1004

References AtProcExit_LocalBuffers(), CheckForBufferLeaks(), and UnlockBuffers().

Referenced by InitBufferManagerAccess().

◆ BgBufferSync()

bool BgBufferSync ( WritebackContext wb_context)

Definition at line 3625 of file bufmgr.c.

3626{
3627 /* info obtained from freelist.c */
3628 int strategy_buf_id;
3629 uint32 strategy_passes;
3630 uint32 recent_alloc;
3631
3632 /*
3633 * Information saved between calls so we can determine the strategy
3634 * point's advance rate and avoid scanning already-cleaned buffers.
3635 */
3636 static bool saved_info_valid = false;
3637 static int prev_strategy_buf_id;
3638 static uint32 prev_strategy_passes;
3639 static int next_to_clean;
3640 static uint32 next_passes;
3641
3642 /* Moving averages of allocation rate and clean-buffer density */
3643 static float smoothed_alloc = 0;
3644 static float smoothed_density = 10.0;
3645
3646 /* Potentially these could be tunables, but for now, not */
3647 float smoothing_samples = 16;
3648 float scan_whole_pool_milliseconds = 120000.0;
3649
3650 /* Used to compute how far we scan ahead */
3651 long strategy_delta;
3652 int bufs_to_lap;
3653 int bufs_ahead;
3654 float scans_per_alloc;
3655 int reusable_buffers_est;
3656 int upcoming_alloc_est;
3657 int min_scan_buffers;
3658
3659 /* Variables for the scanning loop proper */
3660 int num_to_scan;
3661 int num_written;
3662 int reusable_buffers;
3663
3664 /* Variables for final smoothed_density update */
3665 long new_strategy_delta;
3666 uint32 new_recent_alloc;
3667
3668 /*
3669 * Find out where the freelist clock sweep currently is, and how many
3670 * buffer allocations have happened since our last call.
3671 */
3672 strategy_buf_id = StrategySyncStart(&strategy_passes, &recent_alloc);
3673
3674 /* Report buffer alloc counts to pgstat */
3675 PendingBgWriterStats.buf_alloc += recent_alloc;
3676
3677 /*
3678 * If we're not running the LRU scan, just stop after doing the stats
3679 * stuff. We mark the saved state invalid so that we can recover sanely
3680 * if LRU scan is turned back on later.
3681 */
3682 if (bgwriter_lru_maxpages <= 0)
3683 {
3684 saved_info_valid = false;
3685 return true;
3686 }
3687
3688 /*
3689 * Compute strategy_delta = how many buffers have been scanned by the
3690 * clock sweep since last time. If first time through, assume none. Then
3691 * see if we are still ahead of the clock sweep, and if so, how many
3692 * buffers we could scan before we'd catch up with it and "lap" it. Note:
3693 * weird-looking coding of xxx_passes comparisons are to avoid bogus
3694 * behavior when the passes counts wrap around.
3695 */
3696 if (saved_info_valid)
3697 {
3698 int32 passes_delta = strategy_passes - prev_strategy_passes;
3699
3700 strategy_delta = strategy_buf_id - prev_strategy_buf_id;
3701 strategy_delta += (long) passes_delta * NBuffers;
3702
3703 Assert(strategy_delta >= 0);
3704
3705 if ((int32) (next_passes - strategy_passes) > 0)
3706 {
3707 /* we're one pass ahead of the strategy point */
3708 bufs_to_lap = strategy_buf_id - next_to_clean;
3709#ifdef BGW_DEBUG
3710 elog(DEBUG2, "bgwriter ahead: bgw %u-%u strategy %u-%u delta=%ld lap=%d",
3711 next_passes, next_to_clean,
3712 strategy_passes, strategy_buf_id,
3713 strategy_delta, bufs_to_lap);
3714#endif
3715 }
3716 else if (next_passes == strategy_passes &&
3717 next_to_clean >= strategy_buf_id)
3718 {
3719 /* on same pass, but ahead or at least not behind */
3720 bufs_to_lap = NBuffers - (next_to_clean - strategy_buf_id);
3721#ifdef BGW_DEBUG
3722 elog(DEBUG2, "bgwriter ahead: bgw %u-%u strategy %u-%u delta=%ld lap=%d",
3723 next_passes, next_to_clean,
3724 strategy_passes, strategy_buf_id,
3725 strategy_delta, bufs_to_lap);
3726#endif
3727 }
3728 else
3729 {
3730 /*
3731 * We're behind, so skip forward to the strategy point and start
3732 * cleaning from there.
3733 */
3734#ifdef BGW_DEBUG
3735 elog(DEBUG2, "bgwriter behind: bgw %u-%u strategy %u-%u delta=%ld",
3736 next_passes, next_to_clean,
3737 strategy_passes, strategy_buf_id,
3738 strategy_delta);
3739#endif
3740 next_to_clean = strategy_buf_id;
3741 next_passes = strategy_passes;
3742 bufs_to_lap = NBuffers;
3743 }
3744 }
3745 else
3746 {
3747 /*
3748 * Initializing at startup or after LRU scanning had been off. Always
3749 * start at the strategy point.
3750 */
3751#ifdef BGW_DEBUG
3752 elog(DEBUG2, "bgwriter initializing: strategy %u-%u",
3753 strategy_passes, strategy_buf_id);
3754#endif
3755 strategy_delta = 0;
3756 next_to_clean = strategy_buf_id;
3757 next_passes = strategy_passes;
3758 bufs_to_lap = NBuffers;
3759 }
3760
3761 /* Update saved info for next time */
3762 prev_strategy_buf_id = strategy_buf_id;
3763 prev_strategy_passes = strategy_passes;
3764 saved_info_valid = true;
3765
3766 /*
3767 * Compute how many buffers had to be scanned for each new allocation, ie,
3768 * 1/density of reusable buffers, and track a moving average of that.
3769 *
3770 * If the strategy point didn't move, we don't update the density estimate
3771 */
3772 if (strategy_delta > 0 && recent_alloc > 0)
3773 {
3774 scans_per_alloc = (float) strategy_delta / (float) recent_alloc;
3775 smoothed_density += (scans_per_alloc - smoothed_density) /
3776 smoothing_samples;
3777 }
3778
3779 /*
3780 * Estimate how many reusable buffers there are between the current
3781 * strategy point and where we've scanned ahead to, based on the smoothed
3782 * density estimate.
3783 */
3784 bufs_ahead = NBuffers - bufs_to_lap;
3785 reusable_buffers_est = (float) bufs_ahead / smoothed_density;
3786
3787 /*
3788 * Track a moving average of recent buffer allocations. Here, rather than
3789 * a true average we want a fast-attack, slow-decline behavior: we
3790 * immediately follow any increase.
3791 */
3792 if (smoothed_alloc <= (float) recent_alloc)
3793 smoothed_alloc = recent_alloc;
3794 else
3795 smoothed_alloc += ((float) recent_alloc - smoothed_alloc) /
3796 smoothing_samples;
3797
3798 /* Scale the estimate by a GUC to allow more aggressive tuning. */
3799 upcoming_alloc_est = (int) (smoothed_alloc * bgwriter_lru_multiplier);
3800
3801 /*
3802 * If recent_alloc remains at zero for many cycles, smoothed_alloc will
3803 * eventually underflow to zero, and the underflows produce annoying
3804 * kernel warnings on some platforms. Once upcoming_alloc_est has gone to
3805 * zero, there's no point in tracking smaller and smaller values of
3806 * smoothed_alloc, so just reset it to exactly zero to avoid this
3807 * syndrome. It will pop back up as soon as recent_alloc increases.
3808 */
3809 if (upcoming_alloc_est == 0)
3810 smoothed_alloc = 0;
3811
3812 /*
3813 * Even in cases where there's been little or no buffer allocation
3814 * activity, we want to make a small amount of progress through the buffer
3815 * cache so that as many reusable buffers as possible are clean after an
3816 * idle period.
3817 *
3818 * (scan_whole_pool_milliseconds / BgWriterDelay) computes how many times
3819 * the BGW will be called during the scan_whole_pool time; slice the
3820 * buffer pool into that many sections.
3821 */
3822 min_scan_buffers = (int) (NBuffers / (scan_whole_pool_milliseconds / BgWriterDelay));
3823
3824 if (upcoming_alloc_est < (min_scan_buffers + reusable_buffers_est))
3825 {
3826#ifdef BGW_DEBUG
3827 elog(DEBUG2, "bgwriter: alloc_est=%d too small, using min=%d + reusable_est=%d",
3828 upcoming_alloc_est, min_scan_buffers, reusable_buffers_est);
3829#endif
3830 upcoming_alloc_est = min_scan_buffers + reusable_buffers_est;
3831 }
3832
3833 /*
3834 * Now write out dirty reusable buffers, working forward from the
3835 * next_to_clean point, until we have lapped the strategy scan, or cleaned
3836 * enough buffers to match our estimate of the next cycle's allocation
3837 * requirements, or hit the bgwriter_lru_maxpages limit.
3838 */
3839
3840 num_to_scan = bufs_to_lap;
3841 num_written = 0;
3842 reusable_buffers = reusable_buffers_est;
3843
3844 /* Execute the LRU scan */
3845 while (num_to_scan > 0 && reusable_buffers < upcoming_alloc_est)
3846 {
3847 int sync_state = SyncOneBuffer(next_to_clean, true,
3848 wb_context);
3849
3850 if (++next_to_clean >= NBuffers)
3851 {
3852 next_to_clean = 0;
3853 next_passes++;
3854 }
3855 num_to_scan--;
3856
3857 if (sync_state & BUF_WRITTEN)
3858 {
3859 reusable_buffers++;
3860 if (++num_written >= bgwriter_lru_maxpages)
3861 {
3863 break;
3864 }
3865 }
3866 else if (sync_state & BUF_REUSABLE)
3867 reusable_buffers++;
3868 }
3869
3871
3872#ifdef BGW_DEBUG
3873 elog(DEBUG1, "bgwriter: recent_alloc=%u smoothed=%.2f delta=%ld ahead=%d density=%.2f reusable_est=%d upcoming_est=%d scanned=%d wrote=%d reusable=%d",
3874 recent_alloc, smoothed_alloc, strategy_delta, bufs_ahead,
3875 smoothed_density, reusable_buffers_est, upcoming_alloc_est,
3876 bufs_to_lap - num_to_scan,
3877 num_written,
3878 reusable_buffers - reusable_buffers_est);
3879#endif
3880
3881 /*
3882 * Consider the above scan as being like a new allocation scan.
3883 * Characterize its density and update the smoothed one based on it. This
3884 * effectively halves the moving average period in cases where both the
3885 * strategy and the background writer are doing some useful scanning,
3886 * which is helpful because a long memory isn't as desirable on the
3887 * density estimates.
3888 */
3889 new_strategy_delta = bufs_to_lap - num_to_scan;
3890 new_recent_alloc = reusable_buffers - reusable_buffers_est;
3891 if (new_strategy_delta > 0 && new_recent_alloc > 0)
3892 {
3893 scans_per_alloc = (float) new_strategy_delta / (float) new_recent_alloc;
3894 smoothed_density += (scans_per_alloc - smoothed_density) /
3895 smoothing_samples;
3896
3897#ifdef BGW_DEBUG
3898 elog(DEBUG2, "bgwriter: cleaner density alloc=%u scan=%ld density=%.2f new smoothed=%.2f",
3899 new_recent_alloc, new_strategy_delta,
3900 scans_per_alloc, smoothed_density);
3901#endif
3902 }
3903
3904 /* Return true if OK to hibernate */
3905 return (bufs_to_lap == 0 && recent_alloc == 0);
3906}
int BgWriterDelay
Definition: bgwriter.c:58
#define BUF_REUSABLE
Definition: bufmgr.c:81
double bgwriter_lru_multiplier
Definition: bufmgr.c:146
static int SyncOneBuffer(int buf_id, bool skip_recently_used, WritebackContext *wb_context)
Definition: bufmgr.c:3923
int bgwriter_lru_maxpages
Definition: bufmgr.c:145
#define BUF_WRITTEN
Definition: bufmgr.c:80
int32_t int32
Definition: c.h:498
#define DEBUG2
Definition: elog.h:29
#define DEBUG1
Definition: elog.h:30
#define elog(elevel,...)
Definition: elog.h:225
int StrategySyncStart(uint32 *complete_passes, uint32 *num_buf_alloc)
Definition: freelist.c:394
int NBuffers
Definition: globals.c:143
PgStat_BgWriterStats PendingBgWriterStats
PgStat_Counter buf_written_clean
Definition: pgstat.h:239
PgStat_Counter maxwritten_clean
Definition: pgstat.h:240
PgStat_Counter buf_alloc
Definition: pgstat.h:241

References Assert(), bgwriter_lru_maxpages, bgwriter_lru_multiplier, BgWriterDelay, PgStat_BgWriterStats::buf_alloc, BUF_REUSABLE, BUF_WRITTEN, PgStat_BgWriterStats::buf_written_clean, DEBUG1, DEBUG2, elog, PgStat_BgWriterStats::maxwritten_clean, NBuffers, PendingBgWriterStats, StrategySyncStart(), and SyncOneBuffer().

Referenced by BackgroundWriterMain().

◆ buffer_readv_complete()

static pg_attribute_always_inline PgAioResult buffer_readv_complete ( PgAioHandle ioh,
PgAioResult  prior_result,
uint8  cb_data,
bool  is_temp 
)
static

Definition at line 7175 of file bufmgr.c.

7177{
7178 PgAioResult result = prior_result;
7180 uint8 first_error_off = 0;
7181 uint8 first_zeroed_off = 0;
7182 uint8 first_ignored_off = 0;
7183 uint8 error_count = 0;
7184 uint8 zeroed_count = 0;
7185 uint8 ignored_count = 0;
7186 uint8 checkfail_count = 0;
7187 uint64 *io_data;
7188 uint8 handle_data_len;
7189
7190 if (is_temp)
7191 {
7192 Assert(td->smgr.is_temp);
7194 }
7195 else
7196 Assert(!td->smgr.is_temp);
7197
7198 /*
7199 * Iterate over all the buffers affected by this IO and call the
7200 * per-buffer completion function for each buffer.
7201 */
7202 io_data = pgaio_io_get_handle_data(ioh, &handle_data_len);
7203 for (uint8 buf_off = 0; buf_off < handle_data_len; buf_off++)
7204 {
7205 Buffer buf = io_data[buf_off];
7206 bool failed;
7207 bool failed_verification = false;
7208 bool failed_checksum = false;
7209 bool zeroed_buffer = false;
7210 bool ignored_checksum = false;
7211
7213
7214 /*
7215 * If the entire I/O failed on a lower-level, each buffer needs to be
7216 * marked as failed. In case of a partial read, the first few buffers
7217 * may be ok.
7218 */
7219 failed =
7220 prior_result.status == PGAIO_RS_ERROR
7221 || prior_result.result <= buf_off;
7222
7223 buffer_readv_complete_one(td, buf_off, buf, cb_data, failed, is_temp,
7224 &failed_verification,
7225 &failed_checksum,
7226 &ignored_checksum,
7227 &zeroed_buffer);
7228
7229 /*
7230 * Track information about the number of different kinds of error
7231 * conditions across all pages, as there can be multiple pages failing
7232 * verification as part of one IO.
7233 */
7234 if (failed_verification && !zeroed_buffer && error_count++ == 0)
7235 first_error_off = buf_off;
7236 if (zeroed_buffer && zeroed_count++ == 0)
7237 first_zeroed_off = buf_off;
7238 if (ignored_checksum && ignored_count++ == 0)
7239 first_ignored_off = buf_off;
7240 if (failed_checksum)
7241 checkfail_count++;
7242 }
7243
7244 /*
7245 * If the smgr read succeeded [partially] and page verification failed for
7246 * some of the pages, adjust the IO's result state appropriately.
7247 */
7248 if (prior_result.status != PGAIO_RS_ERROR &&
7249 (error_count > 0 || ignored_count > 0 || zeroed_count > 0))
7250 {
7251 buffer_readv_encode_error(&result, is_temp,
7252 zeroed_count > 0, ignored_count > 0,
7253 error_count, zeroed_count, checkfail_count,
7254 first_error_off, first_zeroed_off,
7255 first_ignored_off);
7256 pgaio_result_report(result, td, DEBUG1);
7257 }
7258
7259 /*
7260 * For shared relations this reporting is done in
7261 * shared_buffer_readv_complete_local().
7262 */
7263 if (is_temp && checkfail_count > 0)
7265 checkfail_count);
7266
7267 return result;
7268}
ProcNumber pgaio_io_get_owner(PgAioHandle *ioh)
Definition: aio.c:352
uint64 * pgaio_io_get_handle_data(PgAioHandle *ioh, uint8 *len)
Definition: aio_callback.c:156
void pgaio_result_report(PgAioResult result, const PgAioTargetData *target_data, int elevel)
Definition: aio_callback.c:173
PgAioTargetData * pgaio_io_get_target_data(PgAioHandle *ioh)
Definition: aio_target.c:73
@ PGAIO_RS_ERROR
Definition: aio_types.h:84
static pg_attribute_always_inline void buffer_readv_complete_one(PgAioTargetData *td, uint8 buf_off, Buffer buffer, uint8 flags, bool failed, bool is_temp, bool *buffer_invalid, bool *failed_checksum, bool *ignored_checksum, bool *zeroed_buffer)
Definition: bufmgr.c:7031
static void buffer_readv_encode_error(PgAioResult *result, bool is_temp, bool zeroed_any, bool ignored_any, uint8 error_count, uint8 zeroed_count, uint8 checkfail_count, uint8 first_error_off, uint8 first_zeroed_off, uint8 first_ignored_off)
Definition: bufmgr.c:6936
uint8_t uint8
Definition: c.h:500
uint64_t uint64
Definition: c.h:503
ProcNumber MyProcNumber
Definition: globals.c:91
static char * buf
Definition: pg_test_fsync.c:72
void pgstat_report_checksum_failures_in_db(Oid dboid, int failurecount)
uint32 status
Definition: aio_types.h:108
int32 result
Definition: aio_types.h:113
RelFileLocator rlocator
Definition: aio_types.h:65
struct PgAioTargetData::@124 smgr

References Assert(), buf, buffer_readv_complete_one(), buffer_readv_encode_error(), BufferIsValid(), RelFileLocator::dbOid, DEBUG1, PgAioTargetData::is_temp, MyProcNumber, pgaio_io_get_handle_data(), pgaio_io_get_owner(), pgaio_io_get_target_data(), pgaio_result_report(), PGAIO_RS_ERROR, pgstat_report_checksum_failures_in_db(), PgAioResult::result, PgAioTargetData::rlocator, PgAioTargetData::smgr, and PgAioResult::status.

Referenced by local_buffer_readv_complete(), and shared_buffer_readv_complete().

◆ buffer_readv_complete_one()

static pg_attribute_always_inline void buffer_readv_complete_one ( PgAioTargetData td,
uint8  buf_off,
Buffer  buffer,
uint8  flags,
bool  failed,
bool  is_temp,
bool *  buffer_invalid,
bool *  failed_checksum,
bool *  ignored_checksum,
bool *  zeroed_buffer 
)
static

Definition at line 7031 of file bufmgr.c.

7037{
7038 BufferDesc *buf_hdr = is_temp ?
7039 GetLocalBufferDescriptor(-buffer - 1)
7040 : GetBufferDescriptor(buffer - 1);
7041 BufferTag tag = buf_hdr->tag;
7042 char *bufdata = BufferGetBlock(buffer);
7043 uint32 set_flag_bits;
7044 int piv_flags;
7045
7046 /* check that the buffer is in the expected state for a read */
7047#ifdef USE_ASSERT_CHECKING
7048 {
7049 uint32 buf_state = pg_atomic_read_u32(&buf_hdr->state);
7050
7051 Assert(buf_state & BM_TAG_VALID);
7052 Assert(!(buf_state & BM_VALID));
7053 /* temp buffers don't use BM_IO_IN_PROGRESS */
7054 if (!is_temp)
7055 Assert(buf_state & BM_IO_IN_PROGRESS);
7056 Assert(!(buf_state & BM_DIRTY));
7057 }
7058#endif
7059
7060 *buffer_invalid = false;
7061 *failed_checksum = false;
7062 *ignored_checksum = false;
7063 *zeroed_buffer = false;
7064
7065 /*
7066 * We ask PageIsVerified() to only log the message about checksum errors,
7067 * as the completion might be run in any backend (or IO workers). We will
7068 * report checksum errors in buffer_readv_report().
7069 */
7070 piv_flags = PIV_LOG_LOG;
7071
7072 /* the local zero_damaged_pages may differ from the definer's */
7074 piv_flags |= PIV_IGNORE_CHECKSUM_FAILURE;
7075
7076 /* Check for garbage data. */
7077 if (!failed)
7078 {
7079 /*
7080 * If the buffer is not currently pinned by this backend, e.g. because
7081 * we're completing this IO after an error, the buffer data will have
7082 * been marked as inaccessible when the buffer was unpinned. The AIO
7083 * subsystem holds a pin, but that doesn't prevent the buffer from
7084 * having been marked as inaccessible. The completion might also be
7085 * executed in a different process.
7086 */
7087#ifdef USE_VALGRIND
7088 if (!BufferIsPinned(buffer))
7089 VALGRIND_MAKE_MEM_DEFINED(bufdata, BLCKSZ);
7090#endif
7091
7092 if (!PageIsVerified((Page) bufdata, tag.blockNum, piv_flags,
7093 failed_checksum))
7094 {
7095 if (flags & READ_BUFFERS_ZERO_ON_ERROR)
7096 {
7097 memset(bufdata, 0, BLCKSZ);
7098 *zeroed_buffer = true;
7099 }
7100 else
7101 {
7102 *buffer_invalid = true;
7103 /* mark buffer as having failed */
7104 failed = true;
7105 }
7106 }
7107 else if (*failed_checksum)
7108 *ignored_checksum = true;
7109
7110 /* undo what we did above */
7111#ifdef USE_VALGRIND
7112 if (!BufferIsPinned(buffer))
7113 VALGRIND_MAKE_MEM_NOACCESS(bufdata, BLCKSZ);
7114#endif
7115
7116 /*
7117 * Immediately log a message about the invalid page, but only to the
7118 * server log. The reason to do so immediately is that this may be
7119 * executed in a different backend than the one that originated the
7120 * request. The reason to do so immediately is that the originator
7121 * might not process the query result immediately (because it is busy
7122 * doing another part of query processing) or at all (e.g. if it was
7123 * cancelled or errored out due to another IO also failing). The
7124 * definer of the IO will emit an ERROR or WARNING when processing the
7125 * IO's results
7126 *
7127 * To avoid duplicating the code to emit these log messages, we reuse
7128 * buffer_readv_report().
7129 */
7130 if (*buffer_invalid || *failed_checksum || *zeroed_buffer)
7131 {
7132 PgAioResult result_one = {0};
7133
7134 buffer_readv_encode_error(&result_one, is_temp,
7135 *zeroed_buffer,
7136 *ignored_checksum,
7137 *buffer_invalid,
7138 *zeroed_buffer ? 1 : 0,
7139 *failed_checksum ? 1 : 0,
7140 buf_off, buf_off, buf_off);
7141 pgaio_result_report(result_one, td, LOG_SERVER_ONLY);
7142 }
7143 }
7144
7145 /* Terminate I/O and set BM_VALID. */
7146 set_flag_bits = failed ? BM_IO_ERROR : BM_VALID;
7147 if (is_temp)
7148 TerminateLocalBufferIO(buf_hdr, false, set_flag_bits, true);
7149 else
7150 TerminateBufferIO(buf_hdr, false, set_flag_bits, false, true);
7151
7152 /*
7153 * Call the BUFFER_READ_DONE tracepoint in the callback, even though the
7154 * callback may not be executed in the same backend that called
7155 * BUFFER_READ_START. The alternative would be to defer calling the
7156 * tracepoint to a later point (e.g. the local completion callback for
7157 * shared buffer reads), which seems even less helpful.
7158 */
7159 TRACE_POSTGRESQL_BUFFER_READ_DONE(tag.forkNum,
7160 tag.blockNum,
7161 tag.spcOid,
7162 tag.dbOid,
7163 tag.relNumber,
7165 false);
7166}
static uint32 pg_atomic_read_u32(volatile pg_atomic_uint32 *ptr)
Definition: atomics.h:239
static BufferDesc * GetLocalBufferDescriptor(uint32 id)
#define BufferIsPinned(bufnum)
Definition: bufmgr.c:483
bool PageIsVerified(PageData *page, BlockNumber blkno, int flags, bool *checksum_failure_p)
Definition: bufpage.c:94
#define PIV_LOG_LOG
Definition: bufpage.h:469
PageData * Page
Definition: bufpage.h:82
#define PIV_IGNORE_CHECKSUM_FAILURE
Definition: bufpage.h:470
#define LOG_SERVER_ONLY
Definition: elog.h:32
void TerminateLocalBufferIO(BufferDesc *bufHdr, bool clear_dirty, uint32 set_flag_bits, bool release_aio)
Definition: localbuf.c:560
#define VALGRIND_MAKE_MEM_DEFINED(addr, size)
Definition: memdebug.h:26
#define VALGRIND_MAKE_MEM_NOACCESS(addr, size)
Definition: memdebug.h:27
#define INVALID_PROC_NUMBER
Definition: procnumber.h:26
pg_atomic_uint32 state
RelFileNumber relNumber
ForkNumber forkNum
Oid spcOid

References Assert(), buftag::blockNum, BM_DIRTY, BM_IO_ERROR, BM_IO_IN_PROGRESS, BM_TAG_VALID, BM_VALID, PrivateRefCountEntry::buffer, buffer_readv_encode_error(), BufferGetBlock(), BufferIsPinned, buftag::dbOid, buftag::forkNum, GetBufferDescriptor(), GetLocalBufferDescriptor(), INVALID_PROC_NUMBER, LOG_SERVER_ONLY, MyProcNumber, PageIsVerified(), pg_atomic_read_u32(), pgaio_result_report(), PIV_IGNORE_CHECKSUM_FAILURE, PIV_LOG_LOG, READ_BUFFERS_IGNORE_CHECKSUM_FAILURES, READ_BUFFERS_ZERO_ON_ERROR, buftag::relNumber, buftag::spcOid, BufferDesc::state, BufferDesc::tag, TerminateBufferIO(), TerminateLocalBufferIO(), VALGRIND_MAKE_MEM_DEFINED, and VALGRIND_MAKE_MEM_NOACCESS.

Referenced by buffer_readv_complete().

◆ buffer_readv_decode_error()

static void buffer_readv_decode_error ( PgAioResult  result,
bool *  zeroed_any,
bool *  ignored_any,
uint8 zeroed_or_error_count,
uint8 checkfail_count,
uint8 first_off 
)
inlinestatic

Definition at line 6894 of file bufmgr.c.

6900{
6901 uint32 rem_error = result.error_data;
6902
6903 /* see static asserts in buffer_readv_encode_error */
6904#define READV_COUNT_BITS 7
6905#define READV_COUNT_MASK ((1 << READV_COUNT_BITS) - 1)
6906
6907 *zeroed_any = rem_error & 1;
6908 rem_error >>= 1;
6909
6910 *ignored_any = rem_error & 1;
6911 rem_error >>= 1;
6912
6913 *zeroed_or_error_count = rem_error & READV_COUNT_MASK;
6914 rem_error >>= READV_COUNT_BITS;
6915
6916 *checkfail_count = rem_error & READV_COUNT_MASK;
6917 rem_error >>= READV_COUNT_BITS;
6918
6919 *first_off = rem_error & READV_COUNT_MASK;
6920 rem_error >>= READV_COUNT_BITS;
6921}
#define READV_COUNT_BITS
#define READV_COUNT_MASK
uint32 error_data
Definition: aio_types.h:111

References PgAioResult::error_data, READV_COUNT_BITS, and READV_COUNT_MASK.

Referenced by buffer_readv_encode_error(), buffer_readv_report(), and shared_buffer_readv_complete_local().

◆ buffer_readv_encode_error()

static void buffer_readv_encode_error ( PgAioResult result,
bool  is_temp,
bool  zeroed_any,
bool  ignored_any,
uint8  error_count,
uint8  zeroed_count,
uint8  checkfail_count,
uint8  first_error_off,
uint8  first_zeroed_off,
uint8  first_ignored_off 
)
inlinestatic

Definition at line 6936 of file bufmgr.c.

6946{
6947
6948 uint8 shift = 0;
6949 uint8 zeroed_or_error_count =
6950 error_count > 0 ? error_count : zeroed_count;
6951 uint8 first_off;
6952
6954 "PG_IOV_MAX is bigger than reserved space for error data");
6956 "PGAIO_RESULT_ERROR_BITS is insufficient for buffer_readv");
6957
6958 /*
6959 * We only have space to encode one offset - but luckily that's good
6960 * enough. If there is an error, the error is the interesting offset, same
6961 * with a zeroed buffer vs an ignored buffer.
6962 */
6963 if (error_count > 0)
6964 first_off = first_error_off;
6965 else if (zeroed_count > 0)
6966 first_off = first_zeroed_off;
6967 else
6968 first_off = first_ignored_off;
6969
6970 Assert(!zeroed_any || error_count == 0);
6971
6972 result->error_data = 0;
6973
6974 result->error_data |= zeroed_any << shift;
6975 shift += 1;
6976
6977 result->error_data |= ignored_any << shift;
6978 shift += 1;
6979
6980 result->error_data |= ((uint32) zeroed_or_error_count) << shift;
6981 shift += READV_COUNT_BITS;
6982
6983 result->error_data |= ((uint32) checkfail_count) << shift;
6984 shift += READV_COUNT_BITS;
6985
6986 result->error_data |= ((uint32) first_off) << shift;
6987 shift += READV_COUNT_BITS;
6988
6989 result->id = is_temp ? PGAIO_HCB_LOCAL_BUFFER_READV :
6991
6992 if (error_count > 0)
6993 result->status = PGAIO_RS_ERROR;
6994 else
6995 result->status = PGAIO_RS_WARNING;
6996
6997 /*
6998 * The encoding is complicated enough to warrant cross-checking it against
6999 * the decode function.
7000 */
7001#ifdef USE_ASSERT_CHECKING
7002 {
7003 bool zeroed_any_2,
7004 ignored_any_2;
7005 uint8 zeroed_or_error_count_2,
7006 checkfail_count_2,
7007 first_off_2;
7008
7010 &zeroed_any_2, &ignored_any_2,
7011 &zeroed_or_error_count_2,
7012 &checkfail_count_2,
7013 &first_off_2);
7014 Assert(zeroed_any == zeroed_any_2);
7015 Assert(ignored_any == ignored_any_2);
7016 Assert(zeroed_or_error_count == zeroed_or_error_count_2);
7017 Assert(checkfail_count == checkfail_count_2);
7018 Assert(first_off == first_off_2);
7019 }
7020#endif
7021
7022#undef READV_COUNT_BITS
7023#undef READV_COUNT_MASK
7024}
#define PGAIO_RESULT_ERROR_BITS
Definition: aio_types.h:98
@ PGAIO_RS_WARNING
Definition: aio_types.h:83
static void buffer_readv_decode_error(PgAioResult result, bool *zeroed_any, bool *ignored_any, uint8 *zeroed_or_error_count, uint8 *checkfail_count, uint8 *first_off)
Definition: bufmgr.c:6894
#define StaticAssertStmt(condition, errmessage)
Definition: c.h:909
#define PG_IOV_MAX
Definition: pg_iovec.h:41
uint32 id
Definition: aio_types.h:105

References Assert(), buffer_readv_decode_error(), PgAioResult::error_data, PgAioResult::id, PG_IOV_MAX, PGAIO_HCB_LOCAL_BUFFER_READV, PGAIO_HCB_SHARED_BUFFER_READV, PGAIO_RESULT_ERROR_BITS, PGAIO_RS_ERROR, PGAIO_RS_WARNING, READV_COUNT_BITS, StaticAssertStmt, and PgAioResult::status.

Referenced by buffer_readv_complete(), and buffer_readv_complete_one().

◆ buffer_readv_report()

static void buffer_readv_report ( PgAioResult  result,
const PgAioTargetData td,
int  elevel 
)
static

Definition at line 7278 of file bufmgr.c.

7280{
7281 int nblocks = td->smgr.nblocks;
7282 BlockNumber first = td->smgr.blockNum;
7283 BlockNumber last = first + nblocks - 1;
7284 ProcNumber errProc =
7286 RelPathStr rpath =
7287 relpathbackend(td->smgr.rlocator, errProc, td->smgr.forkNum);
7288 bool zeroed_any,
7289 ignored_any;
7290 uint8 zeroed_or_error_count,
7291 checkfail_count,
7292 first_off;
7293 uint8 affected_count;
7294 const char *msg_one,
7295 *msg_mult,
7296 *det_mult,
7297 *hint_mult;
7298
7299 buffer_readv_decode_error(result, &zeroed_any, &ignored_any,
7300 &zeroed_or_error_count,
7301 &checkfail_count,
7302 &first_off);
7303
7304 /*
7305 * Treat a read that had both zeroed buffers *and* ignored checksums as a
7306 * special case, it's too irregular to be emitted the same way as the
7307 * other cases.
7308 */
7309 if (zeroed_any && ignored_any)
7310 {
7311 Assert(zeroed_any && ignored_any);
7312 Assert(nblocks > 1); /* same block can't be both zeroed and ignored */
7313 Assert(result.status != PGAIO_RS_ERROR);
7314 affected_count = zeroed_or_error_count;
7315
7316 ereport(elevel,
7318 errmsg("zeroing %u page(s) and ignoring %u checksum failure(s) among blocks %u..%u of relation %s",
7319 affected_count, checkfail_count, first, last, rpath.str),
7320 affected_count > 1 ?
7321 errdetail("Block %u held first zeroed page.",
7322 first + first_off) : 0,
7323 errhint("See server log for details about the other %u invalid block(s).",
7324 affected_count + checkfail_count - 1));
7325 return;
7326 }
7327
7328 /*
7329 * The other messages are highly repetitive. To avoid duplicating a long
7330 * and complicated ereport(), gather the translated format strings
7331 * separately and then do one common ereport.
7332 */
7333 if (result.status == PGAIO_RS_ERROR)
7334 {
7335 Assert(!zeroed_any); /* can't have invalid pages when zeroing them */
7336 affected_count = zeroed_or_error_count;
7337 msg_one = _("invalid page in block %u of relation %s");
7338 msg_mult = _("%u invalid pages among blocks %u..%u of relation %s");
7339 det_mult = _("Block %u held first invalid page.");
7340 hint_mult = _("See server log for the other %u invalid block(s).");
7341 }
7342 else if (zeroed_any && !ignored_any)
7343 {
7344 affected_count = zeroed_or_error_count;
7345 msg_one = _("invalid page in block %u of relation %s; zeroing out page");
7346 msg_mult = _("zeroing out %u invalid pages among blocks %u..%u of relation %s");
7347 det_mult = _("Block %u held first zeroed page.");
7348 hint_mult = _("See server log for the other %u zeroed block(s).");
7349 }
7350 else if (!zeroed_any && ignored_any)
7351 {
7352 affected_count = checkfail_count;
7353 msg_one = _("ignoring checksum failure in block %u of relation %s");
7354 msg_mult = _("ignoring %u checksum failures among blocks %u..%u of relation %s");
7355 det_mult = _("Block %u held first ignored page.");
7356 hint_mult = _("See server log for the other %u ignored block(s).");
7357 }
7358 else
7360
7361 ereport(elevel,
7363 affected_count == 1 ?
7364 errmsg_internal(msg_one, first + first_off, rpath.str) :
7365 errmsg_internal(msg_mult, affected_count, first, last, rpath.str),
7366 affected_count > 1 ? errdetail_internal(det_mult, first + first_off) : 0,
7367 affected_count > 1 ? errhint_internal(hint_mult, affected_count - 1) : 0);
7368}
#define pg_unreachable()
Definition: c.h:332
int errmsg_internal(const char *fmt,...)
Definition: elog.c:1158
int errdetail_internal(const char *fmt,...)
Definition: elog.c:1231
int errhint_internal(const char *fmt,...)
Definition: elog.c:1340
int errhint(const char *fmt,...)
Definition: elog.c:1318
#define _(x)
Definition: elog.c:91
#define ERRCODE_DATA_CORRUPTED
Definition: pg_basebackup.c:41
int ProcNumber
Definition: procnumber.h:24
#define relpathbackend(rlocator, backend, forknum)
Definition: relpath.h:141
char str[REL_PATH_STR_MAXLEN+1]
Definition: relpath.h:123
BlockNumber blockNum
Definition: aio_types.h:66
BlockNumber nblocks
Definition: aio_types.h:67
ForkNumber forkNum
Definition: aio_types.h:68

References _, Assert(), PgAioTargetData::blockNum, buffer_readv_decode_error(), ereport, errcode(), ERRCODE_DATA_CORRUPTED, errdetail(), errdetail_internal(), errhint(), errhint_internal(), errmsg(), errmsg_internal(), PgAioTargetData::forkNum, INVALID_PROC_NUMBER, PgAioTargetData::is_temp, MyProcNumber, PgAioTargetData::nblocks, pg_unreachable, PGAIO_RS_ERROR, relpathbackend, PgAioTargetData::rlocator, PgAioTargetData::smgr, PgAioResult::status, and RelPathStr::str.

◆ buffer_stage_common()

static pg_attribute_always_inline void buffer_stage_common ( PgAioHandle ioh,
bool  is_write,
bool  is_temp 
)
static

Definition at line 6785 of file bufmgr.c.

6786{
6787 uint64 *io_data;
6788 uint8 handle_data_len;
6789 PgAioWaitRef io_ref;
6791
6792 io_data = pgaio_io_get_handle_data(ioh, &handle_data_len);
6793
6794 pgaio_io_get_wref(ioh, &io_ref);
6795
6796 /* iterate over all buffers affected by the vectored readv/writev */
6797 for (int i = 0; i < handle_data_len; i++)
6798 {
6799 Buffer buffer = (Buffer) io_data[i];
6800 BufferDesc *buf_hdr = is_temp ?
6801 GetLocalBufferDescriptor(-buffer - 1)
6802 : GetBufferDescriptor(buffer - 1);
6803 uint32 buf_state;
6804
6805 /*
6806 * Check that all the buffers are actually ones that could conceivably
6807 * be done in one IO, i.e. are sequential. This is the last
6808 * buffer-aware code before IO is actually executed and confusion
6809 * about which buffers are targeted by IO can be hard to debug, making
6810 * it worth doing extra-paranoid checks.
6811 */
6812 if (i == 0)
6813 first = buf_hdr->tag;
6814 else
6815 {
6816 Assert(buf_hdr->tag.relNumber == first.relNumber);
6817 Assert(buf_hdr->tag.blockNum == first.blockNum + i);
6818 }
6819
6820 if (is_temp)
6821 buf_state = pg_atomic_read_u32(&buf_hdr->state);
6822 else
6823 buf_state = LockBufHdr(buf_hdr);
6824
6825 /* verify the buffer is in the expected state */
6826 Assert(buf_state & BM_TAG_VALID);
6827 if (is_write)
6828 {
6829 Assert(buf_state & BM_VALID);
6830 Assert(buf_state & BM_DIRTY);
6831 }
6832 else
6833 {
6834 Assert(!(buf_state & BM_VALID));
6835 Assert(!(buf_state & BM_DIRTY));
6836 }
6837
6838 /* temp buffers don't use BM_IO_IN_PROGRESS */
6839 if (!is_temp)
6840 Assert(buf_state & BM_IO_IN_PROGRESS);
6841
6842 Assert(BUF_STATE_GET_REFCOUNT(buf_state) >= 1);
6843
6844 /*
6845 * Reflect that the buffer is now owned by the AIO subsystem.
6846 *
6847 * For local buffers: This can't be done just via LocalRefCount, as
6848 * one might initially think, as this backend could error out while
6849 * AIO is still in progress, releasing all the pins by the backend
6850 * itself.
6851 *
6852 * This pin is released again in TerminateBufferIO().
6853 */
6854 buf_state += BUF_REFCOUNT_ONE;
6855 buf_hdr->io_wref = io_ref;
6856
6857 if (is_temp)
6858 pg_atomic_unlocked_write_u32(&buf_hdr->state, buf_state);
6859 else
6860 UnlockBufHdr(buf_hdr, buf_state);
6861
6862 /*
6863 * Ensure the content lock that prevents buffer modifications while
6864 * the buffer is being written out is not released early due to an
6865 * error.
6866 */
6867 if (is_write && !is_temp)
6868 {
6869 LWLock *content_lock;
6870
6871 content_lock = BufferDescriptorGetContentLock(buf_hdr);
6872
6873 Assert(LWLockHeldByMe(content_lock));
6874
6875 /*
6876 * Lock is now owned by AIO subsystem.
6877 */
6878 LWLockDisown(content_lock);
6879 }
6880
6881 /*
6882 * Stop tracking this buffer via the resowner - the AIO system now
6883 * keeps track.
6884 */
6885 if (!is_temp)
6887 }
6888}
static void pg_atomic_unlocked_write_u32(volatile pg_atomic_uint32 *ptr, uint32 val)
Definition: atomics.h:295
#define BUF_REFCOUNT_ONE
Definition: buf_internals.h:51
static LWLock * BufferDescriptorGetContentLock(const BufferDesc *bdesc)
static void ResourceOwnerForgetBufferIO(ResourceOwner owner, Buffer buffer)
#define BUF_STATE_GET_REFCOUNT(state)
Definition: buf_internals.h:59
#define PG_USED_FOR_ASSERTS_ONLY
Definition: c.h:224
bool LWLockHeldByMe(LWLock *lock)
Definition: lwlock.c:1985
void LWLockDisown(LWLock *lock)
Definition: lwlock.c:1891
PgAioWaitRef io_wref
Definition: lwlock.h:42

References Assert(), buftag::blockNum, BM_DIRTY, BM_IO_IN_PROGRESS, BM_TAG_VALID, BM_VALID, BUF_REFCOUNT_ONE, BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), CurrentResourceOwner, GetBufferDescriptor(), GetLocalBufferDescriptor(), i, BufferDesc::io_wref, LockBufHdr(), LWLockDisown(), LWLockHeldByMe(), pg_atomic_read_u32(), pg_atomic_unlocked_write_u32(), PG_USED_FOR_ASSERTS_ONLY, pgaio_io_get_handle_data(), pgaio_io_get_wref(), buftag::relNumber, ResourceOwnerForgetBufferIO(), BufferDesc::state, BufferDesc::tag, and UnlockBufHdr().

Referenced by local_buffer_readv_stage(), and shared_buffer_readv_stage().

◆ BufferAlloc()

static pg_attribute_always_inline BufferDesc * BufferAlloc ( SMgrRelation  smgr,
char  relpersistence,
ForkNumber  forkNum,
BlockNumber  blockNum,
BufferAccessStrategy  strategy,
bool *  foundPtr,
IOContext  io_context 
)
inlinestatic

Definition at line 2005 of file bufmgr.c.

2009{
2010 BufferTag newTag; /* identity of requested block */
2011 uint32 newHash; /* hash value for newTag */
2012 LWLock *newPartitionLock; /* buffer partition lock for it */
2013 int existing_buf_id;
2014 Buffer victim_buffer;
2015 BufferDesc *victim_buf_hdr;
2016 uint32 victim_buf_state;
2017
2018 /* Make sure we will have room to remember the buffer pin */
2021
2022 /* create a tag so we can lookup the buffer */
2023 InitBufferTag(&newTag, &smgr->smgr_rlocator.locator, forkNum, blockNum);
2024
2025 /* determine its hash code and partition lock ID */
2026 newHash = BufTableHashCode(&newTag);
2027 newPartitionLock = BufMappingPartitionLock(newHash);
2028
2029 /* see if the block is in the buffer pool already */
2030 LWLockAcquire(newPartitionLock, LW_SHARED);
2031 existing_buf_id = BufTableLookup(&newTag, newHash);
2032 if (existing_buf_id >= 0)
2033 {
2034 BufferDesc *buf;
2035 bool valid;
2036
2037 /*
2038 * Found it. Now, pin the buffer so no one can steal it from the
2039 * buffer pool, and check to see if the correct data has been loaded
2040 * into the buffer.
2041 */
2042 buf = GetBufferDescriptor(existing_buf_id);
2043
2044 valid = PinBuffer(buf, strategy);
2045
2046 /* Can release the mapping lock as soon as we've pinned it */
2047 LWLockRelease(newPartitionLock);
2048
2049 *foundPtr = true;
2050
2051 if (!valid)
2052 {
2053 /*
2054 * We can only get here if (a) someone else is still reading in
2055 * the page, (b) a previous read attempt failed, or (c) someone
2056 * called StartReadBuffers() but not yet WaitReadBuffers().
2057 */
2058 *foundPtr = false;
2059 }
2060
2061 return buf;
2062 }
2063
2064 /*
2065 * Didn't find it in the buffer pool. We'll have to initialize a new
2066 * buffer. Remember to unlock the mapping lock while doing the work.
2067 */
2068 LWLockRelease(newPartitionLock);
2069
2070 /*
2071 * Acquire a victim buffer. Somebody else might try to do the same, we
2072 * don't hold any conflicting locks. If so we'll have to undo our work
2073 * later.
2074 */
2075 victim_buffer = GetVictimBuffer(strategy, io_context);
2076 victim_buf_hdr = GetBufferDescriptor(victim_buffer - 1);
2077
2078 /*
2079 * Try to make a hashtable entry for the buffer under its new tag. If
2080 * somebody else inserted another buffer for the tag, we'll release the
2081 * victim buffer we acquired and use the already inserted one.
2082 */
2083 LWLockAcquire(newPartitionLock, LW_EXCLUSIVE);
2084 existing_buf_id = BufTableInsert(&newTag, newHash, victim_buf_hdr->buf_id);
2085 if (existing_buf_id >= 0)
2086 {
2087 BufferDesc *existing_buf_hdr;
2088 bool valid;
2089
2090 /*
2091 * Got a collision. Someone has already done what we were about to do.
2092 * We'll just handle this as if it were found in the buffer pool in
2093 * the first place. First, give up the buffer we were planning to
2094 * use.
2095 *
2096 * We could do this after releasing the partition lock, but then we'd
2097 * have to call ResourceOwnerEnlarge() & ReservePrivateRefCountEntry()
2098 * before acquiring the lock, for the rare case of such a collision.
2099 */
2100 UnpinBuffer(victim_buf_hdr);
2101
2102 /*
2103 * The victim buffer we acquired previously is clean and unused, let
2104 * it be found again quickly
2105 */
2106 StrategyFreeBuffer(victim_buf_hdr);
2107
2108 /* remaining code should match code at top of routine */
2109
2110 existing_buf_hdr = GetBufferDescriptor(existing_buf_id);
2111
2112 valid = PinBuffer(existing_buf_hdr, strategy);
2113
2114 /* Can release the mapping lock as soon as we've pinned it */
2115 LWLockRelease(newPartitionLock);
2116
2117 *foundPtr = true;
2118
2119 if (!valid)
2120 {
2121 /*
2122 * We can only get here if (a) someone else is still reading in
2123 * the page, (b) a previous read attempt failed, or (c) someone
2124 * called StartReadBuffers() but not yet WaitReadBuffers().
2125 */
2126 *foundPtr = false;
2127 }
2128
2129 return existing_buf_hdr;
2130 }
2131
2132 /*
2133 * Need to lock the buffer header too in order to change its tag.
2134 */
2135 victim_buf_state = LockBufHdr(victim_buf_hdr);
2136
2137 /* some sanity checks while we hold the buffer header lock */
2138 Assert(BUF_STATE_GET_REFCOUNT(victim_buf_state) == 1);
2139 Assert(!(victim_buf_state & (BM_TAG_VALID | BM_VALID | BM_DIRTY | BM_IO_IN_PROGRESS)));
2140
2141 victim_buf_hdr->tag = newTag;
2142
2143 /*
2144 * Make sure BM_PERMANENT is set for buffers that must be written at every
2145 * checkpoint. Unlogged buffers only need to be written at shutdown
2146 * checkpoints, except for their "init" forks, which need to be treated
2147 * just like permanent relations.
2148 */
2149 victim_buf_state |= BM_TAG_VALID | BUF_USAGECOUNT_ONE;
2150 if (relpersistence == RELPERSISTENCE_PERMANENT || forkNum == INIT_FORKNUM)
2151 victim_buf_state |= BM_PERMANENT;
2152
2153 UnlockBufHdr(victim_buf_hdr, victim_buf_state);
2154
2155 LWLockRelease(newPartitionLock);
2156
2157 /*
2158 * Buffer contents are currently invalid.
2159 */
2160 *foundPtr = false;
2161
2162 return victim_buf_hdr;
2163}
static void InitBufferTag(BufferTag *tag, const RelFileLocator *rlocator, ForkNumber forkNum, BlockNumber blockNum)
#define BM_PERMANENT
Definition: buf_internals.h:77
#define BUF_USAGECOUNT_ONE
Definition: buf_internals.h:54
static LWLock * BufMappingPartitionLock(uint32 hashcode)
int BufTableLookup(BufferTag *tagPtr, uint32 hashcode)
Definition: buf_table.c:90
uint32 BufTableHashCode(BufferTag *tagPtr)
Definition: buf_table.c:78
int BufTableInsert(BufferTag *tagPtr, uint32 hashcode, int buf_id)
Definition: buf_table.c:118
static bool PinBuffer(BufferDesc *buf, BufferAccessStrategy strategy)
Definition: bufmgr.c:3072
static Buffer GetVictimBuffer(BufferAccessStrategy strategy, IOContext io_context)
Definition: bufmgr.c:2350
static void ReservePrivateRefCountEntry(void)
Definition: bufmgr.c:259
static void UnpinBuffer(BufferDesc *buf)
Definition: bufmgr.c:3264
void StrategyFreeBuffer(BufferDesc *buf)
Definition: freelist.c:363
bool LWLockAcquire(LWLock *lock, LWLockMode mode)
Definition: lwlock.c:1182
void LWLockRelease(LWLock *lock)
Definition: lwlock.c:1902
@ LW_SHARED
Definition: lwlock.h:115
@ LW_EXCLUSIVE
Definition: lwlock.h:114
@ INIT_FORKNUM
Definition: relpath.h:61
void ResourceOwnerEnlarge(ResourceOwner owner)
Definition: resowner.c:452

References Assert(), BM_DIRTY, BM_IO_IN_PROGRESS, BM_PERMANENT, BM_TAG_VALID, BM_VALID, buf, BufferDesc::buf_id, BUF_STATE_GET_REFCOUNT, BUF_USAGECOUNT_ONE, BufMappingPartitionLock(), BufTableHashCode(), BufTableInsert(), BufTableLookup(), CurrentResourceOwner, GetBufferDescriptor(), GetVictimBuffer(), INIT_FORKNUM, InitBufferTag(), RelFileLocatorBackend::locator, LockBufHdr(), LW_EXCLUSIVE, LW_SHARED, LWLockAcquire(), LWLockRelease(), PinBuffer(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), SMgrRelationData::smgr_rlocator, StrategyFreeBuffer(), BufferDesc::tag, UnlockBufHdr(), and UnpinBuffer().

Referenced by PinBufferForBlock().

◆ BufferGetBlockNumber()

BlockNumber BufferGetBlockNumber ( Buffer  buffer)

Definition at line 4229 of file bufmgr.c.

4230{
4231 BufferDesc *bufHdr;
4232
4233 Assert(BufferIsPinned(buffer));
4234
4235 if (BufferIsLocal(buffer))
4236 bufHdr = GetLocalBufferDescriptor(-buffer - 1);
4237 else
4238 bufHdr = GetBufferDescriptor(buffer - 1);
4239
4240 /* pinned, so OK to read tag without spinlock */
4241 return bufHdr->tag.blockNum;
4242}
#define BufferIsLocal(buffer)
Definition: buf.h:37

References Assert(), buftag::blockNum, PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsPinned, GetBufferDescriptor(), GetLocalBufferDescriptor(), and BufferDesc::tag.

Referenced by _bt_binsrch_insert(), _bt_bottomupdel_pass(), _bt_check_unique(), _bt_checkpage(), _bt_delitems_delete(), _bt_delitems_vacuum(), _bt_doinsert(), _bt_finish_split(), _bt_getroot(), _bt_insert_parent(), _bt_insertonpg(), _bt_mark_page_halfdead(), _bt_moveright(), _bt_newlevel(), _bt_pagedel(), _bt_readpage(), _bt_restore_meta(), _bt_search(), _bt_simpledel_pass(), _bt_split(), _bt_unlink_halfdead_page(), _hash_addovflpage(), _hash_checkpage(), _hash_doinsert(), _hash_first(), _hash_freeovflpage(), _hash_getnewbuf(), _hash_readnext(), _hash_readpage(), _hash_splitbucket(), allocNewBuffer(), AsyncReadBuffers(), BitmapHeapScanNextBlock(), blinsert(), BloomInitMetapage(), brin_doinsert(), brin_doupdate(), brin_getinsertbuffer(), brin_initialize_empty_new_buffer(), brin_page_cleanup(), brin_xlog_insert_update(), brinbuild(), brinGetTupleForHeapBlock(), btvacuumpage(), check_index_page(), CheckReadBuffersOperation(), collect_corrupt_items(), collectMatchBitmap(), createPostingTree(), dataBeginPlaceToPageLeaf(), dataPrepareDownlink(), doPickSplit(), entryPrepareDownlink(), fill_seq_fork_with_data(), ginEntryInsert(), ginFindParents(), ginFinishSplit(), ginPlaceToPage(), ginRedoDeleteListPages(), ginRedoUpdateMetapage(), ginScanToDelete(), gistbufferinginserttuples(), gistbuild(), gistcheckpage(), gistdeletepage(), gistformdownlink(), gistinserttuples(), gistMemorizeAllDownlinks(), gistplacetopage(), gistRelocateBuildBuffersOnSplit(), gistScanPage(), gistvacuumpage(), hash_xlog_add_ovfl_page(), heap_delete(), heap_fetch_next_buffer(), heap_hot_search_buffer(), heap_insert(), heap_multi_insert(), heap_page_is_all_visible(), heap_page_prune_and_freeze(), heap_prepare_pagescan(), heap_update(), heap_xlog_confirm(), heap_xlog_lock(), heapam_scan_analyze_next_block(), heapgettup(), heapgettup_pagemode(), index_compute_xid_horizon_for_tuples(), lazy_scan_heap(), lazy_scan_noprune(), lazy_scan_prune(), lazy_vacuum_heap_rel(), makeSublist(), moveLeafs(), moveRightIfItNeeded(), pgstathashindex(), ReadBufferBI(), RelationAddBlocks(), RelationCopyStorageUsingBuffer(), RelationGetBufferForTuple(), RelationPutHeapTuple(), revmap_get_buffer(), revmap_physical_extend(), ScanSourceDatabasePgClassPage(), spgAddNodeAction(), spgbuild(), spgdoinsert(), SpGistSetLastUsedPage(), spgSplitNodeAction(), spgvacuumpage(), spgWalk(), StartReadBuffersImpl(), startScanEntry(), terminate_brin_buildstate(), vacuumLeafPage(), verify_heapam(), visibilitymap_clear(), visibilitymap_get_status(), visibilitymap_pin(), visibilitymap_pin_ok(), and visibilitymap_set().

◆ BufferGetLSNAtomic()

XLogRecPtr BufferGetLSNAtomic ( Buffer  buffer)

Definition at line 4491 of file bufmgr.c.

4492{
4493 char *page = BufferGetPage(buffer);
4494 BufferDesc *bufHdr;
4495 XLogRecPtr lsn;
4496 uint32 buf_state;
4497
4498 /*
4499 * If we don't need locking for correctness, fastpath out.
4500 */
4501 if (!XLogHintBitIsNeeded() || BufferIsLocal(buffer))
4502 return PageGetLSN(page);
4503
4504 /* Make sure we've got a real buffer, and that we hold a pin on it. */
4505 Assert(BufferIsValid(buffer));
4506 Assert(BufferIsPinned(buffer));
4507
4508 bufHdr = GetBufferDescriptor(buffer - 1);
4509 buf_state = LockBufHdr(bufHdr);
4510 lsn = PageGetLSN(page);
4511 UnlockBufHdr(bufHdr, buf_state);
4512
4513 return lsn;
4514}
static Page BufferGetPage(Buffer buffer)
Definition: bufmgr.h:417
static XLogRecPtr PageGetLSN(const PageData *page)
Definition: bufpage.h:386
#define XLogHintBitIsNeeded()
Definition: xlog.h:120
uint64 XLogRecPtr
Definition: xlogdefs.h:21

References Assert(), PrivateRefCountEntry::buffer, BufferGetPage(), BufferIsLocal, BufferIsPinned, BufferIsValid(), GetBufferDescriptor(), LockBufHdr(), PageGetLSN(), UnlockBufHdr(), and XLogHintBitIsNeeded.

Referenced by _bt_killitems(), _bt_readpage(), gin_check_parent_keys_consistency(), gistdoinsert(), gistFindPath(), gistkillitems(), gistScanPage(), SetHintBits(), and XLogSaveBufferForHint().

◆ BufferGetTag()

void BufferGetTag ( Buffer  buffer,
RelFileLocator rlocator,
ForkNumber forknum,
BlockNumber blknum 
)

Definition at line 4250 of file bufmgr.c.

4252{
4253 BufferDesc *bufHdr;
4254
4255 /* Do the same checks as BufferGetBlockNumber. */
4256 Assert(BufferIsPinned(buffer));
4257
4258 if (BufferIsLocal(buffer))
4259 bufHdr = GetLocalBufferDescriptor(-buffer - 1);
4260 else
4261 bufHdr = GetBufferDescriptor(buffer - 1);
4262
4263 /* pinned, so OK to read tag without spinlock */
4264 *rlocator = BufTagGetRelFileLocator(&bufHdr->tag);
4265 *forknum = BufTagGetForkNum(&bufHdr->tag);
4266 *blknum = bufHdr->tag.blockNum;
4267}

References Assert(), buftag::blockNum, PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsPinned, BufTagGetForkNum(), BufTagGetRelFileLocator(), GetBufferDescriptor(), GetLocalBufferDescriptor(), and BufferDesc::tag.

Referenced by fsm_search_avail(), ginRedoInsertEntry(), heap_inplace_update_and_unlock(), log_newpage_buffer(), ResolveCminCmaxDuringDecoding(), XLogRegisterBuffer(), and XLogSaveBufferForHint().

◆ BufferIsDirty()

bool BufferIsDirty ( Buffer  buffer)

Definition at line 2919 of file bufmgr.c.

2920{
2921 BufferDesc *bufHdr;
2922
2923 Assert(BufferIsPinned(buffer));
2924
2925 if (BufferIsLocal(buffer))
2926 {
2927 int bufid = -buffer - 1;
2928
2929 bufHdr = GetLocalBufferDescriptor(bufid);
2930 /* Content locks are not maintained for local buffers. */
2931 }
2932 else
2933 {
2934 bufHdr = GetBufferDescriptor(buffer - 1);
2936 LW_EXCLUSIVE));
2937 }
2938
2939 return pg_atomic_read_u32(&bufHdr->state) & BM_DIRTY;
2940}
bool LWLockHeldByMeInMode(LWLock *lock, LWLockMode mode)
Definition: lwlock.c:2029

References Assert(), BM_DIRTY, PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsPinned, GetBufferDescriptor(), GetLocalBufferDescriptor(), LW_EXCLUSIVE, LWLockHeldByMeInMode(), pg_atomic_read_u32(), and BufferDesc::state.

Referenced by XLogRegisterBuffer().

◆ BufferIsExclusiveLocked()

bool BufferIsExclusiveLocked ( Buffer  buffer)

Definition at line 2891 of file bufmgr.c.

2892{
2893 BufferDesc *bufHdr;
2894
2895 Assert(BufferIsPinned(buffer));
2896
2897 if (BufferIsLocal(buffer))
2898 {
2899 /* Content locks are not maintained for local buffers. */
2900 return true;
2901 }
2902 else
2903 {
2904 bufHdr = GetBufferDescriptor(buffer - 1);
2906 LW_EXCLUSIVE);
2907 }
2908}

References Assert(), PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsPinned, GetBufferDescriptor(), LW_EXCLUSIVE, and LWLockHeldByMeInMode().

Referenced by XLogRegisterBuffer().

◆ BufferIsPermanent()

bool BufferIsPermanent ( Buffer  buffer)

Definition at line 4461 of file bufmgr.c.

4462{
4463 BufferDesc *bufHdr;
4464
4465 /* Local buffers are used only for temp relations. */
4466 if (BufferIsLocal(buffer))
4467 return false;
4468
4469 /* Make sure we've got a real buffer, and that we hold a pin on it. */
4470 Assert(BufferIsValid(buffer));
4471 Assert(BufferIsPinned(buffer));
4472
4473 /*
4474 * BM_PERMANENT can't be changed while we hold a pin on the buffer, so we
4475 * need not bother with the buffer header spinlock. Even if someone else
4476 * changes the buffer header state while we're doing this, the state is
4477 * changed atomically, so we'll read the old value or the new value, but
4478 * not random garbage.
4479 */
4480 bufHdr = GetBufferDescriptor(buffer - 1);
4481 return (pg_atomic_read_u32(&bufHdr->state) & BM_PERMANENT) != 0;
4482}

References Assert(), BM_PERMANENT, PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsPinned, BufferIsValid(), GetBufferDescriptor(), pg_atomic_read_u32(), and BufferDesc::state.

Referenced by SetHintBits().

◆ BufferSync()

static void BufferSync ( int  flags)
static

Definition at line 3349 of file bufmgr.c.

3350{
3351 uint32 buf_state;
3352 int buf_id;
3353 int num_to_scan;
3354 int num_spaces;
3355 int num_processed;
3356 int num_written;
3357 CkptTsStatus *per_ts_stat = NULL;
3358 Oid last_tsid;
3359 binaryheap *ts_heap;
3360 int i;
3361 int mask = BM_DIRTY;
3362 WritebackContext wb_context;
3363
3364 /*
3365 * Unless this is a shutdown checkpoint or we have been explicitly told,
3366 * we write only permanent, dirty buffers. But at shutdown or end of
3367 * recovery, we write all dirty buffers.
3368 */
3371 mask |= BM_PERMANENT;
3372
3373 /*
3374 * Loop over all buffers, and mark the ones that need to be written with
3375 * BM_CHECKPOINT_NEEDED. Count them as we go (num_to_scan), so that we
3376 * can estimate how much work needs to be done.
3377 *
3378 * This allows us to write only those pages that were dirty when the
3379 * checkpoint began, and not those that get dirtied while it proceeds.
3380 * Whenever a page with BM_CHECKPOINT_NEEDED is written out, either by us
3381 * later in this function, or by normal backends or the bgwriter cleaning
3382 * scan, the flag is cleared. Any buffer dirtied after this point won't
3383 * have the flag set.
3384 *
3385 * Note that if we fail to write some buffer, we may leave buffers with
3386 * BM_CHECKPOINT_NEEDED still set. This is OK since any such buffer would
3387 * certainly need to be written for the next checkpoint attempt, too.
3388 */
3389 num_to_scan = 0;
3390 for (buf_id = 0; buf_id < NBuffers; buf_id++)
3391 {
3392 BufferDesc *bufHdr = GetBufferDescriptor(buf_id);
3393
3394 /*
3395 * Header spinlock is enough to examine BM_DIRTY, see comment in
3396 * SyncOneBuffer.
3397 */
3398 buf_state = LockBufHdr(bufHdr);
3399
3400 if ((buf_state & mask) == mask)
3401 {
3402 CkptSortItem *item;
3403
3404 buf_state |= BM_CHECKPOINT_NEEDED;
3405
3406 item = &CkptBufferIds[num_to_scan++];
3407 item->buf_id = buf_id;
3408 item->tsId = bufHdr->tag.spcOid;
3409 item->relNumber = BufTagGetRelNumber(&bufHdr->tag);
3410 item->forkNum = BufTagGetForkNum(&bufHdr->tag);
3411 item->blockNum = bufHdr->tag.blockNum;
3412 }
3413
3414 UnlockBufHdr(bufHdr, buf_state);
3415
3416 /* Check for barrier events in case NBuffers is large. */
3419 }
3420
3421 if (num_to_scan == 0)
3422 return; /* nothing to do */
3423
3425
3426 TRACE_POSTGRESQL_BUFFER_SYNC_START(NBuffers, num_to_scan);
3427
3428 /*
3429 * Sort buffers that need to be written to reduce the likelihood of random
3430 * IO. The sorting is also important for the implementation of balancing
3431 * writes between tablespaces. Without balancing writes we'd potentially
3432 * end up writing to the tablespaces one-by-one; possibly overloading the
3433 * underlying system.
3434 */
3435 sort_checkpoint_bufferids(CkptBufferIds, num_to_scan);
3436
3437 num_spaces = 0;
3438
3439 /*
3440 * Allocate progress status for each tablespace with buffers that need to
3441 * be flushed. This requires the to-be-flushed array to be sorted.
3442 */
3443 last_tsid = InvalidOid;
3444 for (i = 0; i < num_to_scan; i++)
3445 {
3446 CkptTsStatus *s;
3447 Oid cur_tsid;
3448
3449 cur_tsid = CkptBufferIds[i].tsId;
3450
3451 /*
3452 * Grow array of per-tablespace status structs, every time a new
3453 * tablespace is found.
3454 */
3455 if (last_tsid == InvalidOid || last_tsid != cur_tsid)
3456 {
3457 Size sz;
3458
3459 num_spaces++;
3460
3461 /*
3462 * Not worth adding grow-by-power-of-2 logic here - even with a
3463 * few hundred tablespaces this should be fine.
3464 */
3465 sz = sizeof(CkptTsStatus) * num_spaces;
3466
3467 if (per_ts_stat == NULL)
3468 per_ts_stat = (CkptTsStatus *) palloc(sz);
3469 else
3470 per_ts_stat = (CkptTsStatus *) repalloc(per_ts_stat, sz);
3471
3472 s = &per_ts_stat[num_spaces - 1];
3473 memset(s, 0, sizeof(*s));
3474 s->tsId = cur_tsid;
3475
3476 /*
3477 * The first buffer in this tablespace. As CkptBufferIds is sorted
3478 * by tablespace all (s->num_to_scan) buffers in this tablespace
3479 * will follow afterwards.
3480 */
3481 s->index = i;
3482
3483 /*
3484 * progress_slice will be determined once we know how many buffers
3485 * are in each tablespace, i.e. after this loop.
3486 */
3487
3488 last_tsid = cur_tsid;
3489 }
3490 else
3491 {
3492 s = &per_ts_stat[num_spaces - 1];
3493 }
3494
3495 s->num_to_scan++;
3496
3497 /* Check for barrier events. */
3500 }
3501
3502 Assert(num_spaces > 0);
3503
3504 /*
3505 * Build a min-heap over the write-progress in the individual tablespaces,
3506 * and compute how large a portion of the total progress a single
3507 * processed buffer is.
3508 */
3509 ts_heap = binaryheap_allocate(num_spaces,
3511 NULL);
3512
3513 for (i = 0; i < num_spaces; i++)
3514 {
3515 CkptTsStatus *ts_stat = &per_ts_stat[i];
3516
3517 ts_stat->progress_slice = (float8) num_to_scan / ts_stat->num_to_scan;
3518
3519 binaryheap_add_unordered(ts_heap, PointerGetDatum(ts_stat));
3520 }
3521
3522 binaryheap_build(ts_heap);
3523
3524 /*
3525 * Iterate through to-be-checkpointed buffers and write the ones (still)
3526 * marked with BM_CHECKPOINT_NEEDED. The writes are balanced between
3527 * tablespaces; otherwise the sorting would lead to only one tablespace
3528 * receiving writes at a time, making inefficient use of the hardware.
3529 */
3530 num_processed = 0;
3531 num_written = 0;
3532 while (!binaryheap_empty(ts_heap))
3533 {
3534 BufferDesc *bufHdr = NULL;
3535 CkptTsStatus *ts_stat = (CkptTsStatus *)
3537
3538 buf_id = CkptBufferIds[ts_stat->index].buf_id;
3539 Assert(buf_id != -1);
3540
3541 bufHdr = GetBufferDescriptor(buf_id);
3542
3543 num_processed++;
3544
3545 /*
3546 * We don't need to acquire the lock here, because we're only looking
3547 * at a single bit. It's possible that someone else writes the buffer
3548 * and clears the flag right after we check, but that doesn't matter
3549 * since SyncOneBuffer will then do nothing. However, there is a
3550 * further race condition: it's conceivable that between the time we
3551 * examine the bit here and the time SyncOneBuffer acquires the lock,
3552 * someone else not only wrote the buffer but replaced it with another
3553 * page and dirtied it. In that improbable case, SyncOneBuffer will
3554 * write the buffer though we didn't need to. It doesn't seem worth
3555 * guarding against this, though.
3556 */
3558 {
3559 if (SyncOneBuffer(buf_id, false, &wb_context) & BUF_WRITTEN)
3560 {
3561 TRACE_POSTGRESQL_BUFFER_SYNC_WRITTEN(buf_id);
3563 num_written++;
3564 }
3565 }
3566
3567 /*
3568 * Measure progress independent of actually having to flush the buffer
3569 * - otherwise writing become unbalanced.
3570 */
3571 ts_stat->progress += ts_stat->progress_slice;
3572 ts_stat->num_scanned++;
3573 ts_stat->index++;
3574
3575 /* Have all the buffers from the tablespace been processed? */
3576 if (ts_stat->num_scanned == ts_stat->num_to_scan)
3577 {
3578 binaryheap_remove_first(ts_heap);
3579 }
3580 else
3581 {
3582 /* update heap with the new progress */
3583 binaryheap_replace_first(ts_heap, PointerGetDatum(ts_stat));
3584 }
3585
3586 /*
3587 * Sleep to throttle our I/O rate.
3588 *
3589 * (This will check for barrier events even if it doesn't sleep.)
3590 */
3591 CheckpointWriteDelay(flags, (double) num_processed / num_to_scan);
3592 }
3593
3594 /*
3595 * Issue all pending flushes. Only checkpointer calls BufferSync(), so
3596 * IOContext will always be IOCONTEXT_NORMAL.
3597 */
3599
3600 pfree(per_ts_stat);
3601 per_ts_stat = NULL;
3602 binaryheap_free(ts_heap);
3603
3604 /*
3605 * Update checkpoint statistics. As noted above, this doesn't include
3606 * buffers written by other backends or bgwriter scan.
3607 */
3608 CheckpointStats.ckpt_bufs_written += num_written;
3609
3610 TRACE_POSTGRESQL_BUFFER_SYNC_DONE(NBuffers, num_written, num_to_scan);
3611}
void binaryheap_build(binaryheap *heap)
Definition: binaryheap.c:138
void binaryheap_replace_first(binaryheap *heap, bh_node_type d)
Definition: binaryheap.c:255
bh_node_type binaryheap_first(binaryheap *heap)
Definition: binaryheap.c:177
bh_node_type binaryheap_remove_first(binaryheap *heap)
Definition: binaryheap.c:192
void binaryheap_free(binaryheap *heap)
Definition: binaryheap.c:75
void binaryheap_add_unordered(binaryheap *heap, bh_node_type d)
Definition: binaryheap.c:116
binaryheap * binaryheap_allocate(int capacity, binaryheap_comparator compare, void *arg)
Definition: binaryheap.c:39
#define binaryheap_empty(h)
Definition: binaryheap.h:65
CkptSortItem * CkptBufferIds
Definition: buf_init.c:25
static RelFileNumber BufTagGetRelNumber(const BufferTag *tag)
#define BM_CHECKPOINT_NEEDED
Definition: buf_internals.h:76
static int ts_ckpt_progress_comparator(Datum a, Datum b, void *arg)
Definition: bufmgr.c:6376
int checkpoint_flush_after
Definition: bufmgr.c:178
void WritebackContextInit(WritebackContext *context, int *max_pending)
Definition: bufmgr.c:6399
void IssuePendingWritebacks(WritebackContext *wb_context, IOContext io_context)
Definition: bufmgr.c:6461
struct CkptTsStatus CkptTsStatus
double float8
Definition: c.h:601
size_t Size
Definition: c.h:576
void CheckpointWriteDelay(int flags, double progress)
Definition: checkpointer.c:773
volatile sig_atomic_t ProcSignalBarrierPending
Definition: globals.c:40
void * repalloc(void *pointer, Size size)
Definition: mcxt.c:2172
void pfree(void *pointer)
Definition: mcxt.c:2152
void * palloc(Size size)
Definition: mcxt.c:1945
PgStat_CheckpointerStats PendingCheckpointerStats
static Datum PointerGetDatum(const void *X)
Definition: postgres.h:327
static Pointer DatumGetPointer(Datum X)
Definition: postgres.h:317
#define InvalidOid
Definition: postgres_ext.h:35
unsigned int Oid
Definition: postgres_ext.h:30
void ProcessProcSignalBarrier(void)
Definition: procsignal.c:499
int ckpt_bufs_written
Definition: xlog.h:167
ForkNumber forkNum
RelFileNumber relNumber
BlockNumber blockNum
float8 progress_slice
Definition: bufmgr.c:119
int index
Definition: bufmgr.c:127
int num_scanned
Definition: bufmgr.c:124
float8 progress
Definition: bufmgr.c:118
int num_to_scan
Definition: bufmgr.c:122
Oid tsId
Definition: bufmgr.c:109
PgStat_Counter buffers_written
Definition: pgstat.h:263
CheckpointStatsData CheckpointStats
Definition: xlog.c:209
#define CHECKPOINT_END_OF_RECOVERY
Definition: xlog.h:140
#define CHECKPOINT_FLUSH_ALL
Definition: xlog.h:143
#define CHECKPOINT_IS_SHUTDOWN
Definition: xlog.h:139

References Assert(), binaryheap_add_unordered(), binaryheap_allocate(), binaryheap_build(), binaryheap_empty, binaryheap_first(), binaryheap_free(), binaryheap_remove_first(), binaryheap_replace_first(), buftag::blockNum, CkptSortItem::blockNum, BM_CHECKPOINT_NEEDED, BM_DIRTY, BM_PERMANENT, CkptSortItem::buf_id, BUF_WRITTEN, PgStat_CheckpointerStats::buffers_written, BufTagGetForkNum(), BufTagGetRelNumber(), CHECKPOINT_END_OF_RECOVERY, checkpoint_flush_after, CHECKPOINT_FLUSH_ALL, CHECKPOINT_IS_SHUTDOWN, CheckpointStats, CheckpointWriteDelay(), CheckpointStatsData::ckpt_bufs_written, CkptBufferIds, DatumGetPointer(), CkptSortItem::forkNum, GetBufferDescriptor(), i, CkptTsStatus::index, InvalidOid, IOCONTEXT_NORMAL, IssuePendingWritebacks(), LockBufHdr(), NBuffers, CkptTsStatus::num_scanned, CkptTsStatus::num_to_scan, palloc(), PendingCheckpointerStats, pfree(), pg_atomic_read_u32(), PointerGetDatum(), ProcessProcSignalBarrier(), ProcSignalBarrierPending, CkptTsStatus::progress, CkptTsStatus::progress_slice, CkptSortItem::relNumber, repalloc(), buftag::spcOid, BufferDesc::state, SyncOneBuffer(), BufferDesc::tag, ts_ckpt_progress_comparator(), CkptTsStatus::tsId, CkptSortItem::tsId, UnlockBufHdr(), and WritebackContextInit().

Referenced by CheckPointBuffers().

◆ buffertag_comparator()

static int buffertag_comparator ( const BufferTag ba,
const BufferTag bb 
)
inlinestatic

Definition at line 6311 of file bufmgr.c.

6312{
6313 int ret;
6314 RelFileLocator rlocatora;
6315 RelFileLocator rlocatorb;
6316
6317 rlocatora = BufTagGetRelFileLocator(ba);
6318 rlocatorb = BufTagGetRelFileLocator(bb);
6319
6320 ret = rlocator_comparator(&rlocatora, &rlocatorb);
6321
6322 if (ret != 0)
6323 return ret;
6324
6325 if (BufTagGetForkNum(ba) < BufTagGetForkNum(bb))
6326 return -1;
6327 if (BufTagGetForkNum(ba) > BufTagGetForkNum(bb))
6328 return 1;
6329
6330 if (ba->blockNum < bb->blockNum)
6331 return -1;
6332 if (ba->blockNum > bb->blockNum)
6333 return 1;
6334
6335 return 0;
6336}
static int rlocator_comparator(const void *p1, const void *p2)
Definition: bufmgr.c:6230

References buftag::blockNum, BufTagGetForkNum(), BufTagGetRelFileLocator(), and rlocator_comparator().

◆ CheckBufferIsPinnedOnce()

void CheckBufferIsPinnedOnce ( Buffer  buffer)

Definition at line 5652 of file bufmgr.c.

5653{
5654 if (BufferIsLocal(buffer))
5655 {
5656 if (LocalRefCount[-buffer - 1] != 1)
5657 elog(ERROR, "incorrect local pin count: %d",
5658 LocalRefCount[-buffer - 1]);
5659 }
5660 else
5661 {
5662 if (GetPrivateRefCount(buffer) != 1)
5663 elog(ERROR, "incorrect local pin count: %d",
5664 GetPrivateRefCount(buffer));
5665 }
5666}
#define ERROR
Definition: elog.h:39

References PrivateRefCountEntry::buffer, BufferIsLocal, elog, ERROR, GetPrivateRefCount(), and LocalRefCount.

Referenced by GetVictimBuffer(), lazy_scan_heap(), and LockBufferForCleanup().

◆ CheckForBufferLeaks()

static void CheckForBufferLeaks ( void  )
static

Definition at line 4065 of file bufmgr.c.

4066{
4067#ifdef USE_ASSERT_CHECKING
4068 int RefCountErrors = 0;
4070 int i;
4071 char *s;
4072
4073 /* check the array */
4074 for (i = 0; i < REFCOUNT_ARRAY_ENTRIES; i++)
4075 {
4076 res = &PrivateRefCountArray[i];
4077
4078 if (res->buffer != InvalidBuffer)
4079 {
4081 elog(WARNING, "buffer refcount leak: %s", s);
4082 pfree(s);
4083
4084 RefCountErrors++;
4085 }
4086 }
4087
4088 /* if necessary search the hash */
4090 {
4091 HASH_SEQ_STATUS hstat;
4092
4094 while ((res = (PrivateRefCountEntry *) hash_seq_search(&hstat)) != NULL)
4095 {
4097 elog(WARNING, "buffer refcount leak: %s", s);
4098 pfree(s);
4099 RefCountErrors++;
4100 }
4101 }
4102
4103 Assert(RefCountErrors == 0);
4104#endif
4105}
#define InvalidBuffer
Definition: buf.h:25
char * DebugPrintBufferRefcount(Buffer buffer)
Definition: bufmgr.c:4172
#define REFCOUNT_ARRAY_ENTRIES
Definition: bufmgr.c:100
static struct PrivateRefCountEntry PrivateRefCountArray[REFCOUNT_ARRAY_ENTRIES]
Definition: bufmgr.c:215
static HTAB * PrivateRefCountHash
Definition: bufmgr.c:216
void * hash_seq_search(HASH_SEQ_STATUS *status)
Definition: dynahash.c:1421
void hash_seq_init(HASH_SEQ_STATUS *status, HTAB *hashp)
Definition: dynahash.c:1386

References Assert(), PrivateRefCountEntry::buffer, DebugPrintBufferRefcount(), elog, hash_seq_init(), hash_seq_search(), i, InvalidBuffer, pfree(), PrivateRefCountArray, PrivateRefCountHash, PrivateRefCountOverflowed, REFCOUNT_ARRAY_ENTRIES, and WARNING.

Referenced by AtEOXact_Buffers(), and AtProcExit_Buffers().

◆ CheckPointBuffers()

void CheckPointBuffers ( int  flags)

Definition at line 4215 of file bufmgr.c.

4216{
4217 BufferSync(flags);
4218}
static void BufferSync(int flags)
Definition: bufmgr.c:3349

References BufferSync().

Referenced by CheckPointGuts().

◆ CheckReadBuffersOperation()

static void CheckReadBuffersOperation ( ReadBuffersOperation operation,
bool  is_complete 
)
static

Definition at line 1532 of file bufmgr.c.

1533{
1534#ifdef USE_ASSERT_CHECKING
1535 Assert(operation->nblocks_done <= operation->nblocks);
1536 Assert(!is_complete || operation->nblocks == operation->nblocks_done);
1537
1538 for (int i = 0; i < operation->nblocks; i++)
1539 {
1540 Buffer buffer = operation->buffers[i];
1541 BufferDesc *buf_hdr = BufferIsLocal(buffer) ?
1542 GetLocalBufferDescriptor(-buffer - 1) :
1543 GetBufferDescriptor(buffer - 1);
1544
1545 Assert(BufferGetBlockNumber(buffer) == operation->blocknum + i);
1547
1548 if (i < operation->nblocks_done)
1550 }
1551#endif
1552}

References Assert(), ReadBuffersOperation::blocknum, BM_TAG_VALID, BM_VALID, PrivateRefCountEntry::buffer, BufferGetBlockNumber(), BufferIsLocal, ReadBuffersOperation::buffers, GetBufferDescriptor(), GetLocalBufferDescriptor(), i, ReadBuffersOperation::nblocks, ReadBuffersOperation::nblocks_done, pg_atomic_read_u32(), and BufferDesc::state.

Referenced by StartReadBuffersImpl(), and WaitReadBuffers().

◆ ckpt_buforder_comparator()

static int ckpt_buforder_comparator ( const CkptSortItem a,
const CkptSortItem b 
)
inlinestatic

Definition at line 6345 of file bufmgr.c.

6346{
6347 /* compare tablespace */
6348 if (a->tsId < b->tsId)
6349 return -1;
6350 else if (a->tsId > b->tsId)
6351 return 1;
6352 /* compare relation */
6353 if (a->relNumber < b->relNumber)
6354 return -1;
6355 else if (a->relNumber > b->relNumber)
6356 return 1;
6357 /* compare fork */
6358 else if (a->forkNum < b->forkNum)
6359 return -1;
6360 else if (a->forkNum > b->forkNum)
6361 return 1;
6362 /* compare block number */
6363 else if (a->blockNum < b->blockNum)
6364 return -1;
6365 else if (a->blockNum > b->blockNum)
6366 return 1;
6367 /* equal page IDs are unlikely, but not impossible */
6368 return 0;
6369}
int b
Definition: isn.c:74
int a
Definition: isn.c:73

References a, and b.

◆ ConditionalLockBuffer()

bool ConditionalLockBuffer ( Buffer  buffer)

◆ ConditionalLockBufferForCleanup()

bool ConditionalLockBufferForCleanup ( Buffer  buffer)

Definition at line 5853 of file bufmgr.c.

5854{
5855 BufferDesc *bufHdr;
5856 uint32 buf_state,
5857 refcount;
5858
5859 Assert(BufferIsValid(buffer));
5860
5861 /* see AIO related comment in LockBufferForCleanup() */
5862
5863 if (BufferIsLocal(buffer))
5864 {
5865 refcount = LocalRefCount[-buffer - 1];
5866 /* There should be exactly one pin */
5867 Assert(refcount > 0);
5868 if (refcount != 1)
5869 return false;
5870 /* Nobody else to wait for */
5871 return true;
5872 }
5873
5874 /* There should be exactly one local pin */
5875 refcount = GetPrivateRefCount(buffer);
5876 Assert(refcount);
5877 if (refcount != 1)
5878 return false;
5879
5880 /* Try to acquire lock */
5881 if (!ConditionalLockBuffer(buffer))
5882 return false;
5883
5884 bufHdr = GetBufferDescriptor(buffer - 1);
5885 buf_state = LockBufHdr(bufHdr);
5886 refcount = BUF_STATE_GET_REFCOUNT(buf_state);
5887
5888 Assert(refcount > 0);
5889 if (refcount == 1)
5890 {
5891 /* Successfully acquired exclusive lock with pincount 1 */
5892 UnlockBufHdr(bufHdr, buf_state);
5893 return true;
5894 }
5895
5896 /* Failed, so release the lock */
5897 UnlockBufHdr(bufHdr, buf_state);
5899 return false;
5900}
bool ConditionalLockBuffer(Buffer buffer)
Definition: bufmgr.c:5631
void LockBuffer(Buffer buffer, int mode)
Definition: bufmgr.c:5605
#define BUFFER_LOCK_UNLOCK
Definition: bufmgr.h:196

References Assert(), BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BUFFER_LOCK_UNLOCK, BufferIsLocal, BufferIsValid(), ConditionalLockBuffer(), GetBufferDescriptor(), GetPrivateRefCount(), LocalRefCount, LockBuffer(), LockBufHdr(), PrivateRefCountEntry::refcount, and UnlockBufHdr().

Referenced by _hash_finish_split(), _hash_getbuf_with_condlock_cleanup(), heap_page_prune_opt(), and lazy_scan_heap().

◆ CreateAndCopyRelationData()

void CreateAndCopyRelationData ( RelFileLocator  src_rlocator,
RelFileLocator  dst_rlocator,
bool  permanent 
)

Definition at line 5245 of file bufmgr.c.

5247{
5248 char relpersistence;
5249 SMgrRelation src_rel;
5250 SMgrRelation dst_rel;
5251
5252 /* Set the relpersistence. */
5253 relpersistence = permanent ?
5254 RELPERSISTENCE_PERMANENT : RELPERSISTENCE_UNLOGGED;
5255
5256 src_rel = smgropen(src_rlocator, INVALID_PROC_NUMBER);
5257 dst_rel = smgropen(dst_rlocator, INVALID_PROC_NUMBER);
5258
5259 /*
5260 * Create and copy all forks of the relation. During create database we
5261 * have a separate cleanup mechanism which deletes complete database
5262 * directory. Therefore, each individual relation doesn't need to be
5263 * registered for cleanup.
5264 */
5265 RelationCreateStorage(dst_rlocator, relpersistence, false);
5266
5267 /* copy main fork. */
5268 RelationCopyStorageUsingBuffer(src_rlocator, dst_rlocator, MAIN_FORKNUM,
5269 permanent);
5270
5271 /* copy those extra forks that exist */
5272 for (ForkNumber forkNum = MAIN_FORKNUM + 1;
5273 forkNum <= MAX_FORKNUM; forkNum++)
5274 {
5275 if (smgrexists(src_rel, forkNum))
5276 {
5277 smgrcreate(dst_rel, forkNum, false);
5278
5279 /*
5280 * WAL log creation if the relation is persistent, or this is the
5281 * init fork of an unlogged relation.
5282 */
5283 if (permanent || forkNum == INIT_FORKNUM)
5284 log_smgrcreate(&dst_rlocator, forkNum);
5285
5286 /* Copy a fork's data, block by block. */
5287 RelationCopyStorageUsingBuffer(src_rlocator, dst_rlocator, forkNum,
5288 permanent);
5289 }
5290 }
5291}
static void RelationCopyStorageUsingBuffer(RelFileLocator srclocator, RelFileLocator dstlocator, ForkNumber forkNum, bool permanent)
Definition: bufmgr.c:5131
@ MAIN_FORKNUM
Definition: relpath.h:58
#define MAX_FORKNUM
Definition: relpath.h:70
SMgrRelation smgropen(RelFileLocator rlocator, ProcNumber backend)
Definition: smgr.c:240
void smgrcreate(SMgrRelation reln, ForkNumber forknum, bool isRedo)
Definition: smgr.c:481
bool smgrexists(SMgrRelation reln, ForkNumber forknum)
Definition: smgr.c:462
SMgrRelation RelationCreateStorage(RelFileLocator rlocator, char relpersistence, bool register_delete)
Definition: storage.c:122
void log_smgrcreate(const RelFileLocator *rlocator, ForkNumber forkNum)
Definition: storage.c:187

References INIT_FORKNUM, INVALID_PROC_NUMBER, log_smgrcreate(), MAIN_FORKNUM, MAX_FORKNUM, RelationCopyStorageUsingBuffer(), RelationCreateStorage(), smgrcreate(), smgrexists(), and smgropen().

Referenced by CreateDatabaseUsingWalLog().

◆ DebugPrintBufferRefcount()

char * DebugPrintBufferRefcount ( Buffer  buffer)

Definition at line 4172 of file bufmgr.c.

4173{
4174 BufferDesc *buf;
4175 int32 loccount;
4176 char *result;
4177 ProcNumber backend;
4178 uint32 buf_state;
4179
4180 Assert(BufferIsValid(buffer));
4181 if (BufferIsLocal(buffer))
4182 {
4183 buf = GetLocalBufferDescriptor(-buffer - 1);
4184 loccount = LocalRefCount[-buffer - 1];
4185 backend = MyProcNumber;
4186 }
4187 else
4188 {
4189 buf = GetBufferDescriptor(buffer - 1);
4190 loccount = GetPrivateRefCount(buffer);
4191 backend = INVALID_PROC_NUMBER;
4192 }
4193
4194 /* theoretically we should lock the bufhdr here */
4195 buf_state = pg_atomic_read_u32(&buf->state);
4196
4197 result = psprintf("[%03d] (rel=%s, blockNum=%u, flags=0x%x, refcount=%u %d)",
4198 buffer,
4200 BufTagGetForkNum(&buf->tag)).str,
4201 buf->tag.blockNum, buf_state & BUF_FLAG_MASK,
4202 BUF_STATE_GET_REFCOUNT(buf_state), loccount);
4203 return result;
4204}
#define BUF_FLAG_MASK
Definition: buf_internals.h:56
char * psprintf(const char *fmt,...)
Definition: psprintf.c:43

References Assert(), buf, BUF_FLAG_MASK, BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsValid(), BufTagGetForkNum(), BufTagGetRelFileLocator(), GetBufferDescriptor(), GetLocalBufferDescriptor(), GetPrivateRefCount(), INVALID_PROC_NUMBER, LocalRefCount, MyProcNumber, pg_atomic_read_u32(), psprintf(), and relpathbackend.

Referenced by buffer_call_start_io(), buffer_call_terminate_io(), CheckForBufferLeaks(), CheckForLocalBufferLeaks(), and ResOwnerPrintBufferPin().

◆ DropDatabaseBuffers()

void DropDatabaseBuffers ( Oid  dbid)

Definition at line 4893 of file bufmgr.c.

4894{
4895 int i;
4896
4897 /*
4898 * We needn't consider local buffers, since by assumption the target
4899 * database isn't our own.
4900 */
4901
4902 for (i = 0; i < NBuffers; i++)
4903 {
4904 BufferDesc *bufHdr = GetBufferDescriptor(i);
4905 uint32 buf_state;
4906
4907 /*
4908 * As in DropRelationBuffers, an unlocked precheck should be safe and
4909 * saves some cycles.
4910 */
4911 if (bufHdr->tag.dbOid != dbid)
4912 continue;
4913
4914 buf_state = LockBufHdr(bufHdr);
4915 if (bufHdr->tag.dbOid == dbid)
4916 InvalidateBuffer(bufHdr); /* releases spinlock */
4917 else
4918 UnlockBufHdr(bufHdr, buf_state);
4919 }
4920}
static void InvalidateBuffer(BufferDesc *buf)
Definition: bufmgr.c:2183

References buftag::dbOid, GetBufferDescriptor(), i, InvalidateBuffer(), LockBufHdr(), NBuffers, BufferDesc::tag, and UnlockBufHdr().

Referenced by createdb_failure_callback(), dbase_redo(), dropdb(), and movedb().

◆ DropRelationBuffers()

void DropRelationBuffers ( SMgrRelation  smgr_reln,
ForkNumber forkNum,
int  nforks,
BlockNumber firstDelBlock 
)

Definition at line 4538 of file bufmgr.c.

4540{
4541 int i;
4542 int j;
4543 RelFileLocatorBackend rlocator;
4544 BlockNumber nForkBlock[MAX_FORKNUM];
4545 uint64 nBlocksToInvalidate = 0;
4546
4547 rlocator = smgr_reln->smgr_rlocator;
4548
4549 /* If it's a local relation, it's localbuf.c's problem. */
4550 if (RelFileLocatorBackendIsTemp(rlocator))
4551 {
4552 if (rlocator.backend == MyProcNumber)
4553 {
4554 for (j = 0; j < nforks; j++)
4555 DropRelationLocalBuffers(rlocator.locator, forkNum[j],
4556 firstDelBlock[j]);
4557 }
4558 return;
4559 }
4560
4561 /*
4562 * To remove all the pages of the specified relation forks from the buffer
4563 * pool, we need to scan the entire buffer pool but we can optimize it by
4564 * finding the buffers from BufMapping table provided we know the exact
4565 * size of each fork of the relation. The exact size is required to ensure
4566 * that we don't leave any buffer for the relation being dropped as
4567 * otherwise the background writer or checkpointer can lead to a PANIC
4568 * error while flushing buffers corresponding to files that don't exist.
4569 *
4570 * To know the exact size, we rely on the size cached for each fork by us
4571 * during recovery which limits the optimization to recovery and on
4572 * standbys but we can easily extend it once we have shared cache for
4573 * relation size.
4574 *
4575 * In recovery, we cache the value returned by the first lseek(SEEK_END)
4576 * and the future writes keeps the cached value up-to-date. See
4577 * smgrextend. It is possible that the value of the first lseek is smaller
4578 * than the actual number of existing blocks in the file due to buggy
4579 * Linux kernels that might not have accounted for the recent write. But
4580 * that should be fine because there must not be any buffers after that
4581 * file size.
4582 */
4583 for (i = 0; i < nforks; i++)
4584 {
4585 /* Get the number of blocks for a relation's fork */
4586 nForkBlock[i] = smgrnblocks_cached(smgr_reln, forkNum[i]);
4587
4588 if (nForkBlock[i] == InvalidBlockNumber)
4589 {
4590 nBlocksToInvalidate = InvalidBlockNumber;
4591 break;
4592 }
4593
4594 /* calculate the number of blocks to be invalidated */
4595 nBlocksToInvalidate += (nForkBlock[i] - firstDelBlock[i]);
4596 }
4597
4598 /*
4599 * We apply the optimization iff the total number of blocks to invalidate
4600 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
4601 */
4602 if (BlockNumberIsValid(nBlocksToInvalidate) &&
4603 nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
4604 {
4605 for (j = 0; j < nforks; j++)
4606 FindAndDropRelationBuffers(rlocator.locator, forkNum[j],
4607 nForkBlock[j], firstDelBlock[j]);
4608 return;
4609 }
4610
4611 for (i = 0; i < NBuffers; i++)
4612 {
4613 BufferDesc *bufHdr = GetBufferDescriptor(i);
4614 uint32 buf_state;
4615
4616 /*
4617 * We can make this a tad faster by prechecking the buffer tag before
4618 * we attempt to lock the buffer; this saves a lot of lock
4619 * acquisitions in typical cases. It should be safe because the
4620 * caller must have AccessExclusiveLock on the relation, or some other
4621 * reason to be certain that no one is loading new pages of the rel
4622 * into the buffer pool. (Otherwise we might well miss such pages
4623 * entirely.) Therefore, while the tag might be changing while we
4624 * look at it, it can't be changing *to* a value we care about, only
4625 * *away* from such a value. So false negatives are impossible, and
4626 * false positives are safe because we'll recheck after getting the
4627 * buffer lock.
4628 *
4629 * We could check forkNum and blockNum as well as the rlocator, but
4630 * the incremental win from doing so seems small.
4631 */
4632 if (!BufTagMatchesRelFileLocator(&bufHdr->tag, &rlocator.locator))
4633 continue;
4634
4635 buf_state = LockBufHdr(bufHdr);
4636
4637 for (j = 0; j < nforks; j++)
4638 {
4639 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &rlocator.locator) &&
4640 BufTagGetForkNum(&bufHdr->tag) == forkNum[j] &&
4641 bufHdr->tag.blockNum >= firstDelBlock[j])
4642 {
4643 InvalidateBuffer(bufHdr); /* releases spinlock */
4644 break;
4645 }
4646 }
4647 if (j >= nforks)
4648 UnlockBufHdr(bufHdr, buf_state);
4649 }
4650}
#define InvalidBlockNumber
Definition: block.h:33
static bool BlockNumberIsValid(BlockNumber blockNumber)
Definition: block.h:71
static bool BufTagMatchesRelFileLocator(const BufferTag *tag, const RelFileLocator *rlocator)
#define BUF_DROP_FULL_SCAN_THRESHOLD
Definition: bufmgr.c:91
static void FindAndDropRelationBuffers(RelFileLocator rlocator, ForkNumber forkNum, BlockNumber nForkBlock, BlockNumber firstDelBlock)
Definition: bufmgr.c:4832
int j
Definition: isn.c:78
void DropRelationLocalBuffers(RelFileLocator rlocator, ForkNumber forkNum, BlockNumber firstDelBlock)
Definition: localbuf.c:663
#define RelFileLocatorBackendIsTemp(rlocator)
BlockNumber smgrnblocks_cached(SMgrRelation reln, ForkNumber forknum)
Definition: smgr.c:847

References RelFileLocatorBackend::backend, buftag::blockNum, BlockNumberIsValid(), BUF_DROP_FULL_SCAN_THRESHOLD, BufTagGetForkNum(), BufTagMatchesRelFileLocator(), DropRelationLocalBuffers(), FindAndDropRelationBuffers(), GetBufferDescriptor(), i, InvalidateBuffer(), InvalidBlockNumber, j, RelFileLocatorBackend::locator, LockBufHdr(), MAX_FORKNUM, MyProcNumber, NBuffers, RelFileLocatorBackendIsTemp, SMgrRelationData::smgr_rlocator, smgrnblocks_cached(), BufferDesc::tag, and UnlockBufHdr().

Referenced by smgrtruncate().

◆ DropRelationsAllBuffers()

void DropRelationsAllBuffers ( SMgrRelation smgr_reln,
int  nlocators 
)

Definition at line 4661 of file bufmgr.c.

4662{
4663 int i;
4664 int n = 0;
4665 SMgrRelation *rels;
4666 BlockNumber (*block)[MAX_FORKNUM + 1];
4667 uint64 nBlocksToInvalidate = 0;
4668 RelFileLocator *locators;
4669 bool cached = true;
4670 bool use_bsearch;
4671
4672 if (nlocators == 0)
4673 return;
4674
4675 rels = palloc(sizeof(SMgrRelation) * nlocators); /* non-local relations */
4676
4677 /* If it's a local relation, it's localbuf.c's problem. */
4678 for (i = 0; i < nlocators; i++)
4679 {
4680 if (RelFileLocatorBackendIsTemp(smgr_reln[i]->smgr_rlocator))
4681 {
4682 if (smgr_reln[i]->smgr_rlocator.backend == MyProcNumber)
4683 DropRelationAllLocalBuffers(smgr_reln[i]->smgr_rlocator.locator);
4684 }
4685 else
4686 rels[n++] = smgr_reln[i];
4687 }
4688
4689 /*
4690 * If there are no non-local relations, then we're done. Release the
4691 * memory and return.
4692 */
4693 if (n == 0)
4694 {
4695 pfree(rels);
4696 return;
4697 }
4698
4699 /*
4700 * This is used to remember the number of blocks for all the relations
4701 * forks.
4702 */
4703 block = (BlockNumber (*)[MAX_FORKNUM + 1])
4704 palloc(sizeof(BlockNumber) * n * (MAX_FORKNUM + 1));
4705
4706 /*
4707 * We can avoid scanning the entire buffer pool if we know the exact size
4708 * of each of the given relation forks. See DropRelationBuffers.
4709 */
4710 for (i = 0; i < n && cached; i++)
4711 {
4712 for (int j = 0; j <= MAX_FORKNUM; j++)
4713 {
4714 /* Get the number of blocks for a relation's fork. */
4715 block[i][j] = smgrnblocks_cached(rels[i], j);
4716
4717 /* We need to only consider the relation forks that exists. */
4718 if (block[i][j] == InvalidBlockNumber)
4719 {
4720 if (!smgrexists(rels[i], j))
4721 continue;
4722 cached = false;
4723 break;
4724 }
4725
4726 /* calculate the total number of blocks to be invalidated */
4727 nBlocksToInvalidate += block[i][j];
4728 }
4729 }
4730
4731 /*
4732 * We apply the optimization iff the total number of blocks to invalidate
4733 * is below the BUF_DROP_FULL_SCAN_THRESHOLD.
4734 */
4735 if (cached && nBlocksToInvalidate < BUF_DROP_FULL_SCAN_THRESHOLD)
4736 {
4737 for (i = 0; i < n; i++)
4738 {
4739 for (int j = 0; j <= MAX_FORKNUM; j++)
4740 {
4741 /* ignore relation forks that doesn't exist */
4742 if (!BlockNumberIsValid(block[i][j]))
4743 continue;
4744
4745 /* drop all the buffers for a particular relation fork */
4746 FindAndDropRelationBuffers(rels[i]->smgr_rlocator.locator,
4747 j, block[i][j], 0);
4748 }
4749 }
4750
4751 pfree(block);
4752 pfree(rels);
4753 return;
4754 }
4755
4756 pfree(block);
4757 locators = palloc(sizeof(RelFileLocator) * n); /* non-local relations */
4758 for (i = 0; i < n; i++)
4759 locators[i] = rels[i]->smgr_rlocator.locator;
4760
4761 /*
4762 * For low number of relations to drop just use a simple walk through, to
4763 * save the bsearch overhead. The threshold to use is rather a guess than
4764 * an exactly determined value, as it depends on many factors (CPU and RAM
4765 * speeds, amount of shared buffers etc.).
4766 */
4767 use_bsearch = n > RELS_BSEARCH_THRESHOLD;
4768
4769 /* sort the list of rlocators if necessary */
4770 if (use_bsearch)
4771 qsort(locators, n, sizeof(RelFileLocator), rlocator_comparator);
4772
4773 for (i = 0; i < NBuffers; i++)
4774 {
4775 RelFileLocator *rlocator = NULL;
4776 BufferDesc *bufHdr = GetBufferDescriptor(i);
4777 uint32 buf_state;
4778
4779 /*
4780 * As in DropRelationBuffers, an unlocked precheck should be safe and
4781 * saves some cycles.
4782 */
4783
4784 if (!use_bsearch)
4785 {
4786 int j;
4787
4788 for (j = 0; j < n; j++)
4789 {
4790 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &locators[j]))
4791 {
4792 rlocator = &locators[j];
4793 break;
4794 }
4795 }
4796 }
4797 else
4798 {
4799 RelFileLocator locator;
4800
4801 locator = BufTagGetRelFileLocator(&bufHdr->tag);
4802 rlocator = bsearch(&locator,
4803 locators, n, sizeof(RelFileLocator),
4805 }
4806
4807 /* buffer doesn't belong to any of the given relfilelocators; skip it */
4808 if (rlocator == NULL)
4809 continue;
4810
4811 buf_state = LockBufHdr(bufHdr);
4812 if (BufTagMatchesRelFileLocator(&bufHdr->tag, rlocator))
4813 InvalidateBuffer(bufHdr); /* releases spinlock */
4814 else
4815 UnlockBufHdr(bufHdr, buf_state);
4816 }
4817
4818 pfree(locators);
4819 pfree(rels);
4820}
#define RELS_BSEARCH_THRESHOLD
Definition: bufmgr.c:83
if(TABLE==NULL||TABLE_index==NULL)
Definition: isn.c:81
void DropRelationAllLocalBuffers(RelFileLocator rlocator)
Definition: localbuf.c:693
#define qsort(a, b, c, d)
Definition: port.h:479

References BlockNumberIsValid(), BUF_DROP_FULL_SCAN_THRESHOLD, BufTagGetRelFileLocator(), BufTagMatchesRelFileLocator(), DropRelationAllLocalBuffers(), FindAndDropRelationBuffers(), GetBufferDescriptor(), i, if(), InvalidateBuffer(), InvalidBlockNumber, j, LockBufHdr(), MAX_FORKNUM, MyProcNumber, NBuffers, palloc(), pfree(), qsort, RelFileLocatorBackendIsTemp, RELS_BSEARCH_THRESHOLD, rlocator_comparator(), smgrexists(), smgrnblocks_cached(), BufferDesc::tag, and UnlockBufHdr().

Referenced by smgrdounlinkall().

◆ EvictAllUnpinnedBuffers()

void EvictAllUnpinnedBuffers ( int32 buffers_evicted,
int32 buffers_flushed,
int32 buffers_skipped 
)

Definition at line 6678 of file bufmgr.c.

6680{
6681 *buffers_evicted = 0;
6682 *buffers_skipped = 0;
6683 *buffers_flushed = 0;
6684
6685 for (int buf = 1; buf <= NBuffers; buf++)
6686 {
6687 BufferDesc *desc = GetBufferDescriptor(buf - 1);
6688 uint32 buf_state;
6689 bool buffer_flushed;
6690
6691 buf_state = pg_atomic_read_u32(&desc->state);
6692 if (!(buf_state & BM_VALID))
6693 continue;
6694
6697
6698 LockBufHdr(desc);
6699
6700 if (EvictUnpinnedBufferInternal(desc, &buffer_flushed))
6701 (*buffers_evicted)++;
6702 else
6703 (*buffers_skipped)++;
6704
6705 if (buffer_flushed)
6706 (*buffers_flushed)++;
6707 }
6708}
static bool EvictUnpinnedBufferInternal(BufferDesc *desc, bool *buffer_flushed)
Definition: bufmgr.c:6585

References BM_VALID, buf, CurrentResourceOwner, EvictUnpinnedBufferInternal(), GetBufferDescriptor(), LockBufHdr(), NBuffers, pg_atomic_read_u32(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), and BufferDesc::state.

Referenced by pg_buffercache_evict_all().

◆ EvictRelUnpinnedBuffers()

void EvictRelUnpinnedBuffers ( Relation  rel,
int32 buffers_evicted,
int32 buffers_flushed,
int32 buffers_skipped 
)

Definition at line 6726 of file bufmgr.c.

6728{
6730
6731 *buffers_skipped = 0;
6732 *buffers_evicted = 0;
6733 *buffers_flushed = 0;
6734
6735 for (int buf = 1; buf <= NBuffers; buf++)
6736 {
6737 BufferDesc *desc = GetBufferDescriptor(buf - 1);
6738 uint32 buf_state = pg_atomic_read_u32(&(desc->state));
6739 bool buffer_flushed;
6740
6741 /* An unlocked precheck should be safe and saves some cycles. */
6742 if ((buf_state & BM_VALID) == 0 ||
6744 continue;
6745
6746 /* Make sure we can pin the buffer. */
6749
6750 buf_state = LockBufHdr(desc);
6751
6752 /* recheck, could have changed without the lock */
6753 if ((buf_state & BM_VALID) == 0 ||
6755 {
6756 UnlockBufHdr(desc, buf_state);
6757 continue;
6758 }
6759
6760 if (EvictUnpinnedBufferInternal(desc, &buffer_flushed))
6761 (*buffers_evicted)++;
6762 else
6763 (*buffers_skipped)++;
6764
6765 if (buffer_flushed)
6766 (*buffers_flushed)++;
6767 }
6768}
#define RelationUsesLocalBuffers(relation)
Definition: rel.h:648
RelFileLocator rd_locator
Definition: rel.h:57

References Assert(), BM_VALID, buf, BufTagMatchesRelFileLocator(), CurrentResourceOwner, EvictUnpinnedBufferInternal(), GetBufferDescriptor(), LockBufHdr(), NBuffers, pg_atomic_read_u32(), RelationData::rd_locator, RelationUsesLocalBuffers, ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), BufferDesc::state, BufferDesc::tag, and UnlockBufHdr().

Referenced by pg_buffercache_evict_relation().

◆ EvictUnpinnedBuffer()

bool EvictUnpinnedBuffer ( Buffer  buf,
bool *  buffer_flushed 
)

Definition at line 6649 of file bufmgr.c.

6650{
6651 BufferDesc *desc;
6652
6654
6655 /* Make sure we can pin the buffer. */
6658
6659 desc = GetBufferDescriptor(buf - 1);
6660 LockBufHdr(desc);
6661
6662 return EvictUnpinnedBufferInternal(desc, buffer_flushed);
6663}

References Assert(), buf, BufferIsLocal, BufferIsValid(), CurrentResourceOwner, EvictUnpinnedBufferInternal(), GetBufferDescriptor(), LockBufHdr(), ReservePrivateRefCountEntry(), and ResourceOwnerEnlarge().

Referenced by invalidate_rel_block(), modify_rel_block(), and pg_buffercache_evict().

◆ EvictUnpinnedBufferInternal()

static bool EvictUnpinnedBufferInternal ( BufferDesc desc,
bool *  buffer_flushed 
)
static

Definition at line 6585 of file bufmgr.c.

6586{
6587 uint32 buf_state;
6588 bool result;
6589
6590 *buffer_flushed = false;
6591
6592 buf_state = pg_atomic_read_u32(&(desc->state));
6593 Assert(buf_state & BM_LOCKED);
6594
6595 if ((buf_state & BM_VALID) == 0)
6596 {
6597 UnlockBufHdr(desc, buf_state);
6598 return false;
6599 }
6600
6601 /* Check that it's not pinned already. */
6602 if (BUF_STATE_GET_REFCOUNT(buf_state) > 0)
6603 {
6604 UnlockBufHdr(desc, buf_state);
6605 return false;
6606 }
6607
6608 PinBuffer_Locked(desc); /* releases spinlock */
6609
6610 /* If it was dirty, try to clean it once. */
6611 if (buf_state & BM_DIRTY)
6612 {
6615 *buffer_flushed = true;
6617 }
6618
6619 /* This will return false if it becomes dirty or someone else pins it. */
6620 result = InvalidateVictimBuffer(desc);
6621
6622 UnpinBuffer(desc);
6623
6624 return result;
6625}
#define BM_LOCKED
Definition: buf_internals.h:68
static void FlushBuffer(BufferDesc *buf, SMgrRelation reln, IOObject io_object, IOContext io_context)
Definition: bufmgr.c:4289
static void PinBuffer_Locked(BufferDesc *buf)
Definition: bufmgr.c:3183
static bool InvalidateVictimBuffer(BufferDesc *buf_hdr)
Definition: bufmgr.c:2282

References Assert(), BM_DIRTY, BM_LOCKED, BM_VALID, BUF_STATE_GET_REFCOUNT, BufferDescriptorGetContentLock(), FlushBuffer(), InvalidateVictimBuffer(), IOCONTEXT_NORMAL, IOOBJECT_RELATION, LW_SHARED, LWLockAcquire(), LWLockRelease(), pg_atomic_read_u32(), PinBuffer_Locked(), BufferDesc::state, UnlockBufHdr(), and UnpinBuffer().

Referenced by EvictAllUnpinnedBuffers(), EvictRelUnpinnedBuffers(), and EvictUnpinnedBuffer().

◆ ExtendBufferedRel()

Buffer ExtendBufferedRel ( BufferManagerRelation  bmr,
ForkNumber  forkNum,
BufferAccessStrategy  strategy,
uint32  flags 
)

Definition at line 858 of file bufmgr.c.

862{
863 Buffer buf;
864 uint32 extend_by = 1;
865
866 ExtendBufferedRelBy(bmr, forkNum, strategy, flags, extend_by,
867 &buf, &extend_by);
868
869 return buf;
870}
BlockNumber ExtendBufferedRelBy(BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, Buffer *buffers, uint32 *extended_by)
Definition: bufmgr.c:890

References buf, and ExtendBufferedRelBy().

Referenced by _bt_allocbuf(), _hash_getnewbuf(), BloomNewBuffer(), brinbuild(), brinbuildempty(), fill_seq_fork_with_data(), ginbuildempty(), GinNewBuffer(), gistbuildempty(), gistNewBuffer(), ReadBuffer_common(), revmap_physical_extend(), and SpGistNewBuffer().

◆ ExtendBufferedRelBy()

BlockNumber ExtendBufferedRelBy ( BufferManagerRelation  bmr,
ForkNumber  fork,
BufferAccessStrategy  strategy,
uint32  flags,
uint32  extend_by,
Buffer buffers,
uint32 extended_by 
)

Definition at line 890 of file bufmgr.c.

897{
898 Assert((bmr.rel != NULL) != (bmr.smgr != NULL));
899 Assert(bmr.smgr == NULL || bmr.relpersistence != 0);
900 Assert(extend_by > 0);
901
902 if (bmr.smgr == NULL)
903 {
904 bmr.smgr = RelationGetSmgr(bmr.rel);
905 bmr.relpersistence = bmr.rel->rd_rel->relpersistence;
906 }
907
908 return ExtendBufferedRelCommon(bmr, fork, strategy, flags,
909 extend_by, InvalidBlockNumber,
910 buffers, extended_by);
911}
static BlockNumber ExtendBufferedRelCommon(BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, BlockNumber extend_upto, Buffer *buffers, uint32 *extended_by)
Definition: bufmgr.c:2566
static SMgrRelation RelationGetSmgr(Relation rel)
Definition: rel.h:578
struct SMgrRelationData * smgr
Definition: bufmgr.h:104
Form_pg_class rd_rel
Definition: rel.h:111

References Assert(), ExtendBufferedRelCommon(), InvalidBlockNumber, RelationData::rd_rel, BufferManagerRelation::rel, RelationGetSmgr(), BufferManagerRelation::relpersistence, and BufferManagerRelation::smgr.

Referenced by ExtendBufferedRel(), grow_rel(), and RelationAddBlocks().

◆ ExtendBufferedRelCommon()

static BlockNumber ExtendBufferedRelCommon ( BufferManagerRelation  bmr,
ForkNumber  fork,
BufferAccessStrategy  strategy,
uint32  flags,
uint32  extend_by,
BlockNumber  extend_upto,
Buffer buffers,
uint32 extended_by 
)
static

Definition at line 2566 of file bufmgr.c.

2574{
2575 BlockNumber first_block;
2576
2577 TRACE_POSTGRESQL_BUFFER_EXTEND_START(fork,
2582 extend_by);
2583
2584 if (bmr.relpersistence == RELPERSISTENCE_TEMP)
2585 first_block = ExtendBufferedRelLocal(bmr, fork, flags,
2586 extend_by, extend_upto,
2587 buffers, &extend_by);
2588 else
2589 first_block = ExtendBufferedRelShared(bmr, fork, strategy, flags,
2590 extend_by, extend_upto,
2591 buffers, &extend_by);
2592 *extended_by = extend_by;
2593
2594 TRACE_POSTGRESQL_BUFFER_EXTEND_DONE(fork,
2599 *extended_by,
2600 first_block);
2601
2602 return first_block;
2603}
static BlockNumber ExtendBufferedRelShared(BufferManagerRelation bmr, ForkNumber fork, BufferAccessStrategy strategy, uint32 flags, uint32 extend_by, BlockNumber extend_upto, Buffer *buffers, uint32 *extended_by)
Definition: bufmgr.c:2610
BlockNumber ExtendBufferedRelLocal(BufferManagerRelation bmr, ForkNumber fork, uint32 flags, uint32 extend_by, BlockNumber extend_upto, Buffer *buffers, uint32 *extended_by)
Definition: localbuf.c:345

References RelFileLocatorBackend::backend, RelFileLocator::dbOid, ExtendBufferedRelLocal(), ExtendBufferedRelShared(), RelFileLocatorBackend::locator, RelFileLocator::relNumber, BufferManagerRelation::relpersistence, BufferManagerRelation::smgr, SMgrRelationData::smgr_rlocator, and RelFileLocator::spcOid.

Referenced by ExtendBufferedRelBy(), and ExtendBufferedRelTo().

◆ ExtendBufferedRelShared()

static BlockNumber ExtendBufferedRelShared ( BufferManagerRelation  bmr,
ForkNumber  fork,
BufferAccessStrategy  strategy,
uint32  flags,
uint32  extend_by,
BlockNumber  extend_upto,
Buffer buffers,
uint32 extended_by 
)
static

Definition at line 2610 of file bufmgr.c.

2618{
2619 BlockNumber first_block;
2620 IOContext io_context = IOContextForStrategy(strategy);
2621 instr_time io_start;
2622
2623 LimitAdditionalPins(&extend_by);
2624
2625 /*
2626 * Acquire victim buffers for extension without holding extension lock.
2627 * Writing out victim buffers is the most expensive part of extending the
2628 * relation, particularly when doing so requires WAL flushes. Zeroing out
2629 * the buffers is also quite expensive, so do that before holding the
2630 * extension lock as well.
2631 *
2632 * These pages are pinned by us and not valid. While we hold the pin they
2633 * can't be acquired as victim buffers by another backend.
2634 */
2635 for (uint32 i = 0; i < extend_by; i++)
2636 {
2637 Block buf_block;
2638
2639 buffers[i] = GetVictimBuffer(strategy, io_context);
2640 buf_block = BufHdrGetBlock(GetBufferDescriptor(buffers[i] - 1));
2641
2642 /* new buffers are zero-filled */
2643 MemSet(buf_block, 0, BLCKSZ);
2644 }
2645
2646 /*
2647 * Lock relation against concurrent extensions, unless requested not to.
2648 *
2649 * We use the same extension lock for all forks. That's unnecessarily
2650 * restrictive, but currently extensions for forks don't happen often
2651 * enough to make it worth locking more granularly.
2652 *
2653 * Note that another backend might have extended the relation by the time
2654 * we get the lock.
2655 */
2656 if (!(flags & EB_SKIP_EXTENSION_LOCK))
2658
2659 /*
2660 * If requested, invalidate size cache, so that smgrnblocks asks the
2661 * kernel.
2662 */
2663 if (flags & EB_CLEAR_SIZE_CACHE)
2665
2666 first_block = smgrnblocks(bmr.smgr, fork);
2667
2668 /*
2669 * Now that we have the accurate relation size, check if the caller wants
2670 * us to extend to only up to a specific size. If there were concurrent
2671 * extensions, we might have acquired too many buffers and need to release
2672 * them.
2673 */
2674 if (extend_upto != InvalidBlockNumber)
2675 {
2676 uint32 orig_extend_by = extend_by;
2677
2678 if (first_block > extend_upto)
2679 extend_by = 0;
2680 else if ((uint64) first_block + extend_by > extend_upto)
2681 extend_by = extend_upto - first_block;
2682
2683 for (uint32 i = extend_by; i < orig_extend_by; i++)
2684 {
2685 BufferDesc *buf_hdr = GetBufferDescriptor(buffers[i] - 1);
2686
2687 /*
2688 * The victim buffer we acquired previously is clean and unused,
2689 * let it be found again quickly
2690 */
2691 StrategyFreeBuffer(buf_hdr);
2692 UnpinBuffer(buf_hdr);
2693 }
2694
2695 if (extend_by == 0)
2696 {
2697 if (!(flags & EB_SKIP_EXTENSION_LOCK))
2699 *extended_by = extend_by;
2700 return first_block;
2701 }
2702 }
2703
2704 /* Fail if relation is already at maximum possible length */
2705 if ((uint64) first_block + extend_by >= MaxBlockNumber)
2706 ereport(ERROR,
2707 (errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
2708 errmsg("cannot extend relation %s beyond %u blocks",
2709 relpath(bmr.smgr->smgr_rlocator, fork).str,
2710 MaxBlockNumber)));
2711
2712 /*
2713 * Insert buffers into buffer table, mark as IO_IN_PROGRESS.
2714 *
2715 * This needs to happen before we extend the relation, because as soon as
2716 * we do, other backends can start to read in those pages.
2717 */
2718 for (uint32 i = 0; i < extend_by; i++)
2719 {
2720 Buffer victim_buf = buffers[i];
2721 BufferDesc *victim_buf_hdr = GetBufferDescriptor(victim_buf - 1);
2722 BufferTag tag;
2723 uint32 hash;
2724 LWLock *partition_lock;
2725 int existing_id;
2726
2727 /* in case we need to pin an existing buffer below */
2730
2731 InitBufferTag(&tag, &bmr.smgr->smgr_rlocator.locator, fork, first_block + i);
2732 hash = BufTableHashCode(&tag);
2733 partition_lock = BufMappingPartitionLock(hash);
2734
2735 LWLockAcquire(partition_lock, LW_EXCLUSIVE);
2736
2737 existing_id = BufTableInsert(&tag, hash, victim_buf_hdr->buf_id);
2738
2739 /*
2740 * We get here only in the corner case where we are trying to extend
2741 * the relation but we found a pre-existing buffer. This can happen
2742 * because a prior attempt at extending the relation failed, and
2743 * because mdread doesn't complain about reads beyond EOF (when
2744 * zero_damaged_pages is ON) and so a previous attempt to read a block
2745 * beyond EOF could have left a "valid" zero-filled buffer.
2746 * Unfortunately, we have also seen this case occurring because of
2747 * buggy Linux kernels that sometimes return an lseek(SEEK_END) result
2748 * that doesn't account for a recent write. In that situation, the
2749 * pre-existing buffer would contain valid data that we don't want to
2750 * overwrite. Since the legitimate cases should always have left a
2751 * zero-filled buffer, complain if not PageIsNew.
2752 */
2753 if (existing_id >= 0)
2754 {
2755 BufferDesc *existing_hdr = GetBufferDescriptor(existing_id);
2756 Block buf_block;
2757 bool valid;
2758
2759 /*
2760 * Pin the existing buffer before releasing the partition lock,
2761 * preventing it from being evicted.
2762 */
2763 valid = PinBuffer(existing_hdr, strategy);
2764
2765 LWLockRelease(partition_lock);
2766
2767 /*
2768 * The victim buffer we acquired previously is clean and unused,
2769 * let it be found again quickly
2770 */
2771 StrategyFreeBuffer(victim_buf_hdr);
2772 UnpinBuffer(victim_buf_hdr);
2773
2774 buffers[i] = BufferDescriptorGetBuffer(existing_hdr);
2775 buf_block = BufHdrGetBlock(existing_hdr);
2776
2777 if (valid && !PageIsNew((Page) buf_block))
2778 ereport(ERROR,
2779 (errmsg("unexpected data beyond EOF in block %u of relation %s",
2780 existing_hdr->tag.blockNum,
2781 relpath(bmr.smgr->smgr_rlocator, fork).str),
2782 errhint("This has been seen to occur with buggy kernels; consider updating your system.")));
2783
2784 /*
2785 * We *must* do smgr[zero]extend before succeeding, else the page
2786 * will not be reserved by the kernel, and the next P_NEW call
2787 * will decide to return the same page. Clear the BM_VALID bit,
2788 * do StartBufferIO() and proceed.
2789 *
2790 * Loop to handle the very small possibility that someone re-sets
2791 * BM_VALID between our clearing it and StartBufferIO inspecting
2792 * it.
2793 */
2794 do
2795 {
2796 uint32 buf_state = LockBufHdr(existing_hdr);
2797
2798 buf_state &= ~BM_VALID;
2799 UnlockBufHdr(existing_hdr, buf_state);
2800 } while (!StartBufferIO(existing_hdr, true, false));
2801 }
2802 else
2803 {
2804 uint32 buf_state;
2805
2806 buf_state = LockBufHdr(victim_buf_hdr);
2807
2808 /* some sanity checks while we hold the buffer header lock */
2809 Assert(!(buf_state & (BM_VALID | BM_TAG_VALID | BM_DIRTY | BM_JUST_DIRTIED)));
2810 Assert(BUF_STATE_GET_REFCOUNT(buf_state) == 1);
2811
2812 victim_buf_hdr->tag = tag;
2813
2814 buf_state |= BM_TAG_VALID | BUF_USAGECOUNT_ONE;
2815 if (bmr.relpersistence == RELPERSISTENCE_PERMANENT || fork == INIT_FORKNUM)
2816 buf_state |= BM_PERMANENT;
2817
2818 UnlockBufHdr(victim_buf_hdr, buf_state);
2819
2820 LWLockRelease(partition_lock);
2821
2822 /* XXX: could combine the locked operations in it with the above */
2823 StartBufferIO(victim_buf_hdr, true, false);
2824 }
2825 }
2826
2828
2829 /*
2830 * Note: if smgrzeroextend fails, we will end up with buffers that are
2831 * allocated but not marked BM_VALID. The next relation extension will
2832 * still select the same block number (because the relation didn't get any
2833 * longer on disk) and so future attempts to extend the relation will find
2834 * the same buffers (if they have not been recycled) but come right back
2835 * here to try smgrzeroextend again.
2836 *
2837 * We don't need to set checksum for all-zero pages.
2838 */
2839 smgrzeroextend(bmr.smgr, fork, first_block, extend_by, false);
2840
2841 /*
2842 * Release the file-extension lock; it's now OK for someone else to extend
2843 * the relation some more.
2844 *
2845 * We remove IO_IN_PROGRESS after this, as waking up waiting backends can
2846 * take noticeable time.
2847 */
2848 if (!(flags & EB_SKIP_EXTENSION_LOCK))
2850
2852 io_start, 1, extend_by * BLCKSZ);
2853
2854 /* Set BM_VALID, terminate IO, and wake up any waiters */
2855 for (uint32 i = 0; i < extend_by; i++)
2856 {
2857 Buffer buf = buffers[i];
2858 BufferDesc *buf_hdr = GetBufferDescriptor(buf - 1);
2859 bool lock = false;
2860
2861 if (flags & EB_LOCK_FIRST && i == 0)
2862 lock = true;
2863 else if (flags & EB_LOCK_TARGET)
2864 {
2865 Assert(extend_upto != InvalidBlockNumber);
2866 if (first_block + i + 1 == extend_upto)
2867 lock = true;
2868 }
2869
2870 if (lock)
2872
2873 TerminateBufferIO(buf_hdr, false, BM_VALID, true, false);
2874 }
2875
2877
2878 *extended_by = extend_by;
2879
2880 return first_block;
2881}
#define MaxBlockNumber
Definition: block.h:35
#define BM_JUST_DIRTIED
Definition: buf_internals.h:74
static Buffer BufferDescriptorGetBuffer(const BufferDesc *bdesc)
#define BufHdrGetBlock(bufHdr)
Definition: bufmgr.c:72
void LimitAdditionalPins(uint32 *additional_pins)
Definition: bufmgr.c:2548
bool StartBufferIO(BufferDesc *buf, bool forInput, bool nowait)
Definition: bufmgr.c:6043
void * Block
Definition: bufmgr.h:26
@ EB_LOCK_TARGET
Definition: bufmgr.h:93
@ EB_CLEAR_SIZE_CACHE
Definition: bufmgr.h:90
@ EB_SKIP_EXTENSION_LOCK
Definition: bufmgr.h:75
@ EB_LOCK_FIRST
Definition: bufmgr.h:87
static bool PageIsNew(const PageData *page)
Definition: bufpage.h:234
#define MemSet(start, val, len)
Definition: c.h:991
void LockRelationForExtension(Relation relation, LOCKMODE lockmode)
Definition: lmgr.c:424
void UnlockRelationForExtension(Relation relation, LOCKMODE lockmode)
Definition: lmgr.c:474
#define ExclusiveLock
Definition: lockdefs.h:42
@ IOOP_EXTEND
Definition: pgstat.h:311
static unsigned hash(unsigned *uv, int n)
Definition: rege_dfa.c:715
#define relpath(rlocator, forknum)
Definition: relpath.h:150
BlockNumber smgrnblocks(SMgrRelation reln, ForkNumber forknum)
Definition: smgr.c:819
void smgrzeroextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, int nblocks, bool skipFsync)
Definition: smgr.c:649
int64 shared_blks_written
Definition: instrument.h:29
BlockNumber smgr_cached_nblocks[MAX_FORKNUM+1]
Definition: smgr.h:47

References Assert(), buftag::blockNum, BM_DIRTY, BM_JUST_DIRTIED, BM_PERMANENT, BM_TAG_VALID, BM_VALID, buf, BufferDesc::buf_id, BUF_STATE_GET_REFCOUNT, BUF_USAGECOUNT_ONE, BufferDescriptorGetBuffer(), BufferDescriptorGetContentLock(), BufHdrGetBlock, BufMappingPartitionLock(), BufTableHashCode(), BufTableInsert(), CurrentResourceOwner, EB_CLEAR_SIZE_CACHE, EB_LOCK_FIRST, EB_LOCK_TARGET, EB_SKIP_EXTENSION_LOCK, ereport, errcode(), errhint(), errmsg(), ERROR, ExclusiveLock, GetBufferDescriptor(), GetVictimBuffer(), hash(), i, INIT_FORKNUM, InitBufferTag(), InvalidBlockNumber, IOContextForStrategy(), IOOBJECT_RELATION, IOOP_EXTEND, LimitAdditionalPins(), RelFileLocatorBackend::locator, LockBufHdr(), LockRelationForExtension(), LW_EXCLUSIVE, LWLockAcquire(), LWLockRelease(), MaxBlockNumber, MemSet, PageIsNew(), pgBufferUsage, pgstat_count_io_op_time(), pgstat_prepare_io_time(), PinBuffer(), BufferManagerRelation::rel, relpath, BufferManagerRelation::relpersistence, ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), BufferUsage::shared_blks_written, BufferManagerRelation::smgr, SMgrRelationData::smgr_cached_nblocks, SMgrRelationData::smgr_rlocator, smgrnblocks(), smgrzeroextend(), StartBufferIO(), StrategyFreeBuffer(), BufferDesc::tag, TerminateBufferIO(), track_io_timing, UnlockBufHdr(), UnlockRelationForExtension(), and UnpinBuffer().

Referenced by ExtendBufferedRelCommon().

◆ ExtendBufferedRelTo()

Buffer ExtendBufferedRelTo ( BufferManagerRelation  bmr,
ForkNumber  fork,
BufferAccessStrategy  strategy,
uint32  flags,
BlockNumber  extend_to,
ReadBufferMode  mode 
)

Definition at line 922 of file bufmgr.c.

928{
930 uint32 extended_by = 0;
931 Buffer buffer = InvalidBuffer;
932 Buffer buffers[64];
933
934 Assert((bmr.rel != NULL) != (bmr.smgr != NULL));
935 Assert(bmr.smgr == NULL || bmr.relpersistence != 0);
936 Assert(extend_to != InvalidBlockNumber && extend_to > 0);
937
938 if (bmr.smgr == NULL)
939 {
940 bmr.smgr = RelationGetSmgr(bmr.rel);
941 bmr.relpersistence = bmr.rel->rd_rel->relpersistence;
942 }
943
944 /*
945 * If desired, create the file if it doesn't exist. If
946 * smgr_cached_nblocks[fork] is positive then it must exist, no need for
947 * an smgrexists call.
948 */
949 if ((flags & EB_CREATE_FORK_IF_NEEDED) &&
950 (bmr.smgr->smgr_cached_nblocks[fork] == 0 ||
952 !smgrexists(bmr.smgr, fork))
953 {
955
956 /* recheck, fork might have been created concurrently */
957 if (!smgrexists(bmr.smgr, fork))
958 smgrcreate(bmr.smgr, fork, flags & EB_PERFORMING_RECOVERY);
959
961 }
962
963 /*
964 * If requested, invalidate size cache, so that smgrnblocks asks the
965 * kernel.
966 */
967 if (flags & EB_CLEAR_SIZE_CACHE)
969
970 /*
971 * Estimate how many pages we'll need to extend by. This avoids acquiring
972 * unnecessarily many victim buffers.
973 */
974 current_size = smgrnblocks(bmr.smgr, fork);
975
976 /*
977 * Since no-one else can be looking at the page contents yet, there is no
978 * difference between an exclusive lock and a cleanup-strength lock. Note
979 * that we pass the original mode to ReadBuffer_common() below, when
980 * falling back to reading the buffer to a concurrent relation extension.
981 */
983 flags |= EB_LOCK_TARGET;
984
985 while (current_size < extend_to)
986 {
987 uint32 num_pages = lengthof(buffers);
988 BlockNumber first_block;
989
990 if ((uint64) current_size + num_pages > extend_to)
991 num_pages = extend_to - current_size;
992
993 first_block = ExtendBufferedRelCommon(bmr, fork, strategy, flags,
994 num_pages, extend_to,
995 buffers, &extended_by);
996
997 current_size = first_block + extended_by;
998 Assert(num_pages != 0 || current_size >= extend_to);
999
1000 for (uint32 i = 0; i < extended_by; i++)
1001 {
1002 if (first_block + i != extend_to - 1)
1003 ReleaseBuffer(buffers[i]);
1004 else
1005 buffer = buffers[i];
1006 }
1007 }
1008
1009 /*
1010 * It's possible that another backend concurrently extended the relation.
1011 * In that case read the buffer.
1012 *
1013 * XXX: Should we control this via a flag?
1014 */
1015 if (buffer == InvalidBuffer)
1016 {
1017 Assert(extended_by == 0);
1018 buffer = ReadBuffer_common(bmr.rel, bmr.smgr, bmr.relpersistence,
1019 fork, extend_to - 1, mode, strategy);
1020 }
1021
1022 return buffer;
1023}
static Buffer ReadBuffer_common(Relation rel, SMgrRelation smgr, char smgr_persistence, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy)
Definition: bufmgr.c:1193
void ReleaseBuffer(Buffer buffer)
Definition: bufmgr.c:5371
@ EB_PERFORMING_RECOVERY
Definition: bufmgr.h:78
@ EB_CREATE_FORK_IF_NEEDED
Definition: bufmgr.h:84
@ RBM_ZERO_AND_CLEANUP_LOCK
Definition: bufmgr.h:49
@ RBM_ZERO_AND_LOCK
Definition: bufmgr.h:47
#define lengthof(array)
Definition: c.h:759
static PgChecksumMode mode
Definition: pg_checksums.c:55
static int64 current_size
Definition: pg_checksums.c:63

References Assert(), PrivateRefCountEntry::buffer, current_size, EB_CLEAR_SIZE_CACHE, EB_CREATE_FORK_IF_NEEDED, EB_LOCK_TARGET, EB_PERFORMING_RECOVERY, ExclusiveLock, ExtendBufferedRelCommon(), i, InvalidBlockNumber, InvalidBuffer, lengthof, LockRelationForExtension(), mode, RBM_ZERO_AND_CLEANUP_LOCK, RBM_ZERO_AND_LOCK, RelationData::rd_rel, ReadBuffer_common(), BufferManagerRelation::rel, RelationGetSmgr(), ReleaseBuffer(), BufferManagerRelation::relpersistence, BufferManagerRelation::smgr, SMgrRelationData::smgr_cached_nblocks, smgrcreate(), smgrexists(), smgrnblocks(), and UnlockRelationForExtension().

Referenced by fsm_extend(), vm_extend(), and XLogReadBufferExtended().

◆ FindAndDropRelationBuffers()

static void FindAndDropRelationBuffers ( RelFileLocator  rlocator,
ForkNumber  forkNum,
BlockNumber  nForkBlock,
BlockNumber  firstDelBlock 
)
static

Definition at line 4832 of file bufmgr.c.

4835{
4836 BlockNumber curBlock;
4837
4838 for (curBlock = firstDelBlock; curBlock < nForkBlock; curBlock++)
4839 {
4840 uint32 bufHash; /* hash value for tag */
4841 BufferTag bufTag; /* identity of requested block */
4842 LWLock *bufPartitionLock; /* buffer partition lock for it */
4843 int buf_id;
4844 BufferDesc *bufHdr;
4845 uint32 buf_state;
4846
4847 /* create a tag so we can lookup the buffer */
4848 InitBufferTag(&bufTag, &rlocator, forkNum, curBlock);
4849
4850 /* determine its hash code and partition lock ID */
4851 bufHash = BufTableHashCode(&bufTag);
4852 bufPartitionLock = BufMappingPartitionLock(bufHash);
4853
4854 /* Check that it is in the buffer pool. If not, do nothing. */
4855 LWLockAcquire(bufPartitionLock, LW_SHARED);
4856 buf_id = BufTableLookup(&bufTag, bufHash);
4857 LWLockRelease(bufPartitionLock);
4858
4859 if (buf_id < 0)
4860 continue;
4861
4862 bufHdr = GetBufferDescriptor(buf_id);
4863
4864 /*
4865 * We need to lock the buffer header and recheck if the buffer is
4866 * still associated with the same block because the buffer could be
4867 * evicted by some other backend loading blocks for a different
4868 * relation after we release lock on the BufMapping table.
4869 */
4870 buf_state = LockBufHdr(bufHdr);
4871
4872 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &rlocator) &&
4873 BufTagGetForkNum(&bufHdr->tag) == forkNum &&
4874 bufHdr->tag.blockNum >= firstDelBlock)
4875 InvalidateBuffer(bufHdr); /* releases spinlock */
4876 else
4877 UnlockBufHdr(bufHdr, buf_state);
4878 }
4879}

References buftag::blockNum, BufMappingPartitionLock(), BufTableHashCode(), BufTableLookup(), BufTagGetForkNum(), BufTagMatchesRelFileLocator(), GetBufferDescriptor(), InitBufferTag(), InvalidateBuffer(), LockBufHdr(), LW_SHARED, LWLockAcquire(), LWLockRelease(), BufferDesc::tag, and UnlockBufHdr().

Referenced by DropRelationBuffers(), and DropRelationsAllBuffers().

◆ FlushBuffer()

static void FlushBuffer ( BufferDesc buf,
SMgrRelation  reln,
IOObject  io_object,
IOContext  io_context 
)
static

Definition at line 4289 of file bufmgr.c.

4291{
4292 XLogRecPtr recptr;
4293 ErrorContextCallback errcallback;
4294 instr_time io_start;
4295 Block bufBlock;
4296 char *bufToWrite;
4297 uint32 buf_state;
4298
4299 /*
4300 * Try to start an I/O operation. If StartBufferIO returns false, then
4301 * someone else flushed the buffer before we could, so we need not do
4302 * anything.
4303 */
4304 if (!StartBufferIO(buf, false, false))
4305 return;
4306
4307 /* Setup error traceback support for ereport() */
4309 errcallback.arg = buf;
4310 errcallback.previous = error_context_stack;
4311 error_context_stack = &errcallback;
4312
4313 /* Find smgr relation for buffer */
4314 if (reln == NULL)
4316
4317 TRACE_POSTGRESQL_BUFFER_FLUSH_START(BufTagGetForkNum(&buf->tag),
4318 buf->tag.blockNum,
4322
4323 buf_state = LockBufHdr(buf);
4324
4325 /*
4326 * Run PageGetLSN while holding header lock, since we don't have the
4327 * buffer locked exclusively in all cases.
4328 */
4329 recptr = BufferGetLSN(buf);
4330
4331 /* To check if block content changes while flushing. - vadim 01/17/97 */
4332 buf_state &= ~BM_JUST_DIRTIED;
4333 UnlockBufHdr(buf, buf_state);
4334
4335 /*
4336 * Force XLOG flush up to buffer's LSN. This implements the basic WAL
4337 * rule that log updates must hit disk before any of the data-file changes
4338 * they describe do.
4339 *
4340 * However, this rule does not apply to unlogged relations, which will be
4341 * lost after a crash anyway. Most unlogged relation pages do not bear
4342 * LSNs since we never emit WAL records for them, and therefore flushing
4343 * up through the buffer LSN would be useless, but harmless. However,
4344 * GiST indexes use LSNs internally to track page-splits, and therefore
4345 * unlogged GiST pages bear "fake" LSNs generated by
4346 * GetFakeLSNForUnloggedRel. It is unlikely but possible that the fake
4347 * LSN counter could advance past the WAL insertion point; and if it did
4348 * happen, attempting to flush WAL through that location would fail, with
4349 * disastrous system-wide consequences. To make sure that can't happen,
4350 * skip the flush if the buffer isn't permanent.
4351 */
4352 if (buf_state & BM_PERMANENT)
4353 XLogFlush(recptr);
4354
4355 /*
4356 * Now it's safe to write the buffer to disk. Note that no one else should
4357 * have been able to write it, while we were busy with log flushing,
4358 * because we got the exclusive right to perform I/O by setting the
4359 * BM_IO_IN_PROGRESS bit.
4360 */
4361 bufBlock = BufHdrGetBlock(buf);
4362
4363 /*
4364 * Update page checksum if desired. Since we have only shared lock on the
4365 * buffer, other processes might be updating hint bits in it, so we must
4366 * copy the page to private storage if we do checksumming.
4367 */
4368 bufToWrite = PageSetChecksumCopy((Page) bufBlock, buf->tag.blockNum);
4369
4371
4372 /*
4373 * bufToWrite is either the shared buffer or a copy, as appropriate.
4374 */
4375 smgrwrite(reln,
4376 BufTagGetForkNum(&buf->tag),
4377 buf->tag.blockNum,
4378 bufToWrite,
4379 false);
4380
4381 /*
4382 * When a strategy is in use, only flushes of dirty buffers already in the
4383 * strategy ring are counted as strategy writes (IOCONTEXT
4384 * [BULKREAD|BULKWRITE|VACUUM] IOOP_WRITE) for the purpose of IO
4385 * statistics tracking.
4386 *
4387 * If a shared buffer initially added to the ring must be flushed before
4388 * being used, this is counted as an IOCONTEXT_NORMAL IOOP_WRITE.
4389 *
4390 * If a shared buffer which was added to the ring later because the
4391 * current strategy buffer is pinned or in use or because all strategy
4392 * buffers were dirty and rejected (for BAS_BULKREAD operations only)
4393 * requires flushing, this is counted as an IOCONTEXT_NORMAL IOOP_WRITE
4394 * (from_ring will be false).
4395 *
4396 * When a strategy is not in use, the write can only be a "regular" write
4397 * of a dirty shared buffer (IOCONTEXT_NORMAL IOOP_WRITE).
4398 */
4400 IOOP_WRITE, io_start, 1, BLCKSZ);
4401
4403
4404 /*
4405 * Mark the buffer as clean (unless BM_JUST_DIRTIED has become set) and
4406 * end the BM_IO_IN_PROGRESS state.
4407 */
4408 TerminateBufferIO(buf, true, 0, true, false);
4409
4410 TRACE_POSTGRESQL_BUFFER_FLUSH_DONE(BufTagGetForkNum(&buf->tag),
4411 buf->tag.blockNum,
4415
4416 /* Pop the error context stack */
4417 error_context_stack = errcallback.previous;
4418}
#define BufferGetLSN(bufHdr)
Definition: bufmgr.c:73
static void shared_buffer_write_error_callback(void *arg)
Definition: bufmgr.c:6198
char * PageSetChecksumCopy(Page page, BlockNumber blkno)
Definition: bufpage.c:1509
ErrorContextCallback * error_context_stack
Definition: elog.c:95
@ IOOP_WRITE
Definition: pgstat.h:313
static void smgrwrite(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, const void *buffer, bool skipFsync)
Definition: smgr.h:131
struct ErrorContextCallback * previous
Definition: elog.h:296
void(* callback)(void *arg)
Definition: elog.h:297
void XLogFlush(XLogRecPtr record)
Definition: xlog.c:2923

References ErrorContextCallback::arg, BM_PERMANENT, buf, BufferGetLSN, BufHdrGetBlock, BufTagGetForkNum(), BufTagGetRelFileLocator(), ErrorContextCallback::callback, RelFileLocator::dbOid, error_context_stack, INVALID_PROC_NUMBER, IOOBJECT_RELATION, IOOP_WRITE, RelFileLocatorBackend::locator, LockBufHdr(), PageSetChecksumCopy(), pgBufferUsage, pgstat_count_io_op_time(), pgstat_prepare_io_time(), ErrorContextCallback::previous, RelFileLocator::relNumber, BufferUsage::shared_blks_written, shared_buffer_write_error_callback(), SMgrRelationData::smgr_rlocator, smgropen(), smgrwrite(), RelFileLocator::spcOid, StartBufferIO(), TerminateBufferIO(), track_io_timing, UnlockBufHdr(), and XLogFlush().

Referenced by EvictUnpinnedBufferInternal(), FlushDatabaseBuffers(), FlushOneBuffer(), FlushRelationBuffers(), FlushRelationsAllBuffers(), GetVictimBuffer(), and SyncOneBuffer().

◆ FlushDatabaseBuffers()

void FlushDatabaseBuffers ( Oid  dbid)

Definition at line 5309 of file bufmgr.c.

5310{
5311 int i;
5312 BufferDesc *bufHdr;
5313
5314 for (i = 0; i < NBuffers; i++)
5315 {
5316 uint32 buf_state;
5317
5318 bufHdr = GetBufferDescriptor(i);
5319
5320 /*
5321 * As in DropRelationBuffers, an unlocked precheck should be safe and
5322 * saves some cycles.
5323 */
5324 if (bufHdr->tag.dbOid != dbid)
5325 continue;
5326
5327 /* Make sure we can handle the pin */
5330
5331 buf_state = LockBufHdr(bufHdr);
5332 if (bufHdr->tag.dbOid == dbid &&
5333 (buf_state & (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
5334 {
5335 PinBuffer_Locked(bufHdr);
5339 UnpinBuffer(bufHdr);
5340 }
5341 else
5342 UnlockBufHdr(bufHdr, buf_state);
5343 }
5344}

References BM_DIRTY, BM_VALID, BufferDescriptorGetContentLock(), CurrentResourceOwner, buftag::dbOid, FlushBuffer(), GetBufferDescriptor(), i, IOCONTEXT_NORMAL, IOOBJECT_RELATION, LockBufHdr(), LW_SHARED, LWLockAcquire(), LWLockRelease(), NBuffers, PinBuffer_Locked(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), BufferDesc::tag, UnlockBufHdr(), and UnpinBuffer().

Referenced by dbase_redo().

◆ FlushOneBuffer()

void FlushOneBuffer ( Buffer  buffer)

Definition at line 5351 of file bufmgr.c.

5352{
5353 BufferDesc *bufHdr;
5354
5355 /* currently not needed, but no fundamental reason not to support */
5356 Assert(!BufferIsLocal(buffer));
5357
5358 Assert(BufferIsPinned(buffer));
5359
5360 bufHdr = GetBufferDescriptor(buffer - 1);
5361
5363
5365}

References Assert(), PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsPinned, FlushBuffer(), GetBufferDescriptor(), IOCONTEXT_NORMAL, IOOBJECT_RELATION, and LWLockHeldByMe().

Referenced by hash_xlog_init_bitmap_page(), hash_xlog_init_meta_page(), invalidate_rel_block(), and XLogReadBufferForRedoExtended().

◆ FlushRelationBuffers()

void FlushRelationBuffers ( Relation  rel)

Definition at line 4941 of file bufmgr.c.

4942{
4943 int i;
4944 BufferDesc *bufHdr;
4945 SMgrRelation srel = RelationGetSmgr(rel);
4946
4947 if (RelationUsesLocalBuffers(rel))
4948 {
4949 for (i = 0; i < NLocBuffer; i++)
4950 {
4951 uint32 buf_state;
4952
4953 bufHdr = GetLocalBufferDescriptor(i);
4954 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &rel->rd_locator) &&
4955 ((buf_state = pg_atomic_read_u32(&bufHdr->state)) &
4956 (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
4957 {
4958 ErrorContextCallback errcallback;
4959
4960 /* Setup error traceback support for ereport() */
4962 errcallback.arg = bufHdr;
4963 errcallback.previous = error_context_stack;
4964 error_context_stack = &errcallback;
4965
4966 /* Make sure we can handle the pin */
4969
4970 /*
4971 * Pin/unpin mostly to make valgrind work, but it also seems
4972 * like the right thing to do.
4973 */
4974 PinLocalBuffer(bufHdr, false);
4975
4976
4977 FlushLocalBuffer(bufHdr, srel);
4978
4980
4981 /* Pop the error context stack */
4982 error_context_stack = errcallback.previous;
4983 }
4984 }
4985
4986 return;
4987 }
4988
4989 for (i = 0; i < NBuffers; i++)
4990 {
4991 uint32 buf_state;
4992
4993 bufHdr = GetBufferDescriptor(i);
4994
4995 /*
4996 * As in DropRelationBuffers, an unlocked precheck should be safe and
4997 * saves some cycles.
4998 */
4999 if (!BufTagMatchesRelFileLocator(&bufHdr->tag, &rel->rd_locator))
5000 continue;
5001
5002 /* Make sure we can handle the pin */
5005
5006 buf_state = LockBufHdr(bufHdr);
5007 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &rel->rd_locator) &&
5008 (buf_state & (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
5009 {
5010 PinBuffer_Locked(bufHdr);
5014 UnpinBuffer(bufHdr);
5015 }
5016 else
5017 UnlockBufHdr(bufHdr, buf_state);
5018 }
5019}
static void local_buffer_write_error_callback(void *arg)
Definition: bufmgr.c:6214
void FlushLocalBuffer(BufferDesc *bufHdr, SMgrRelation reln)
Definition: localbuf.c:182
void UnpinLocalBuffer(Buffer buffer)
Definition: localbuf.c:832
bool PinLocalBuffer(BufferDesc *buf_hdr, bool adjust_usagecount)
Definition: localbuf.c:796
int NLocBuffer
Definition: localbuf.c:44

References ErrorContextCallback::arg, BM_DIRTY, BM_VALID, BufferDescriptorGetBuffer(), BufferDescriptorGetContentLock(), BufTagMatchesRelFileLocator(), ErrorContextCallback::callback, CurrentResourceOwner, error_context_stack, FlushBuffer(), FlushLocalBuffer(), GetBufferDescriptor(), GetLocalBufferDescriptor(), i, IOCONTEXT_NORMAL, IOOBJECT_RELATION, local_buffer_write_error_callback(), LockBufHdr(), LW_SHARED, LWLockAcquire(), LWLockRelease(), NBuffers, NLocBuffer, pg_atomic_read_u32(), PinBuffer_Locked(), PinLocalBuffer(), ErrorContextCallback::previous, RelationData::rd_locator, RelationGetSmgr(), RelationUsesLocalBuffers, ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), BufferDesc::state, BufferDesc::tag, UnlockBufHdr(), UnpinBuffer(), and UnpinLocalBuffer().

Referenced by fill_seq_with_data(), heapam_relation_copy_data(), and index_copy_data().

◆ FlushRelationsAllBuffers()

void FlushRelationsAllBuffers ( SMgrRelation smgrs,
int  nrels 
)

Definition at line 5031 of file bufmgr.c.

5032{
5033 int i;
5034 SMgrSortArray *srels;
5035 bool use_bsearch;
5036
5037 if (nrels == 0)
5038 return;
5039
5040 /* fill-in array for qsort */
5041 srels = palloc(sizeof(SMgrSortArray) * nrels);
5042
5043 for (i = 0; i < nrels; i++)
5044 {
5045 Assert(!RelFileLocatorBackendIsTemp(smgrs[i]->smgr_rlocator));
5046
5047 srels[i].rlocator = smgrs[i]->smgr_rlocator.locator;
5048 srels[i].srel = smgrs[i];
5049 }
5050
5051 /*
5052 * Save the bsearch overhead for low number of relations to sync. See
5053 * DropRelationsAllBuffers for details.
5054 */
5055 use_bsearch = nrels > RELS_BSEARCH_THRESHOLD;
5056
5057 /* sort the list of SMgrRelations if necessary */
5058 if (use_bsearch)
5059 qsort(srels, nrels, sizeof(SMgrSortArray), rlocator_comparator);
5060
5061 for (i = 0; i < NBuffers; i++)
5062 {
5063 SMgrSortArray *srelent = NULL;
5064 BufferDesc *bufHdr = GetBufferDescriptor(i);
5065 uint32 buf_state;
5066
5067 /*
5068 * As in DropRelationBuffers, an unlocked precheck should be safe and
5069 * saves some cycles.
5070 */
5071
5072 if (!use_bsearch)
5073 {
5074 int j;
5075
5076 for (j = 0; j < nrels; j++)
5077 {
5078 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &srels[j].rlocator))
5079 {
5080 srelent = &srels[j];
5081 break;
5082 }
5083 }
5084 }
5085 else
5086 {
5087 RelFileLocator rlocator;
5088
5089 rlocator = BufTagGetRelFileLocator(&bufHdr->tag);
5090 srelent = bsearch(&rlocator,
5091 srels, nrels, sizeof(SMgrSortArray),
5093 }
5094
5095 /* buffer doesn't belong to any of the given relfilelocators; skip it */
5096 if (srelent == NULL)
5097 continue;
5098
5099 /* Make sure we can handle the pin */
5102
5103 buf_state = LockBufHdr(bufHdr);
5104 if (BufTagMatchesRelFileLocator(&bufHdr->tag, &srelent->rlocator) &&
5105 (buf_state & (BM_VALID | BM_DIRTY)) == (BM_VALID | BM_DIRTY))
5106 {
5107 PinBuffer_Locked(bufHdr);
5111 UnpinBuffer(bufHdr);
5112 }
5113 else
5114 UnlockBufHdr(bufHdr, buf_state);
5115 }
5116
5117 pfree(srels);
5118}
SMgrRelation srel
Definition: bufmgr.c:140
RelFileLocator rlocator
Definition: bufmgr.c:139

References Assert(), BM_DIRTY, BM_VALID, BufferDescriptorGetContentLock(), BufTagGetRelFileLocator(), BufTagMatchesRelFileLocator(), CurrentResourceOwner, FlushBuffer(), GetBufferDescriptor(), i, IOCONTEXT_NORMAL, IOOBJECT_RELATION, j, RelFileLocatorBackend::locator, LockBufHdr(), LW_SHARED, LWLockAcquire(), LWLockRelease(), NBuffers, palloc(), pfree(), PinBuffer_Locked(), qsort, RelFileLocatorBackendIsTemp, RELS_BSEARCH_THRESHOLD, ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), SMgrSortArray::rlocator, rlocator_comparator(), SMgrRelationData::smgr_rlocator, SMgrSortArray::srel, BufferDesc::tag, UnlockBufHdr(), and UnpinBuffer().

Referenced by smgrdosyncall().

◆ ForgetPrivateRefCountEntry()

static void ForgetPrivateRefCountEntry ( PrivateRefCountEntry ref)
static

Definition at line 448 of file bufmgr.c.

449{
450 Assert(ref->refcount == 0);
451
452 if (ref >= &PrivateRefCountArray[0] &&
454 {
455 ref->buffer = InvalidBuffer;
456
457 /*
458 * Mark the just used entry as reserved - in many scenarios that
459 * allows us to avoid ever having to search the array/hash for free
460 * entries.
461 */
463 }
464 else
465 {
466 bool found;
467 Buffer buffer = ref->buffer;
468
470 Assert(found);
473 }
474}
static PrivateRefCountEntry * ReservedRefCountEntry
Definition: bufmgr.c:219
void * hash_search(HTAB *hashp, const void *keyPtr, HASHACTION action, bool *foundPtr)
Definition: dynahash.c:956
@ HASH_REMOVE
Definition: hsearch.h:115

References Assert(), PrivateRefCountEntry::buffer, HASH_REMOVE, hash_search(), InvalidBuffer, PrivateRefCountArray, PrivateRefCountHash, PrivateRefCountOverflowed, PrivateRefCountEntry::refcount, REFCOUNT_ARRAY_ENTRIES, and ReservedRefCountEntry.

Referenced by UnpinBufferNoOwner().

◆ GetAdditionalPinLimit()

uint32 GetAdditionalPinLimit ( void  )

Definition at line 2522 of file bufmgr.c.

2523{
2524 uint32 estimated_pins_held;
2525
2526 /*
2527 * We get the number of "overflowed" pins for free, but don't know the
2528 * number of pins in PrivateRefCountArray. The cost of calculating that
2529 * exactly doesn't seem worth it, so just assume the max.
2530 */
2531 estimated_pins_held = PrivateRefCountOverflowed + REFCOUNT_ARRAY_ENTRIES;
2532
2533 /* Is this backend already holding more than its fair share? */
2534 if (estimated_pins_held > MaxProportionalPins)
2535 return 0;
2536
2537 return MaxProportionalPins - estimated_pins_held;
2538}
static uint32 MaxProportionalPins
Definition: bufmgr.c:221

References MaxProportionalPins, PrivateRefCountOverflowed, and REFCOUNT_ARRAY_ENTRIES.

Referenced by LimitAdditionalPins(), and read_stream_start_pending_read().

◆ GetPinLimit()

uint32 GetPinLimit ( void  )

Definition at line 2510 of file bufmgr.c.

2511{
2512 return MaxProportionalPins;
2513}

References MaxProportionalPins.

Referenced by GetAccessStrategy(), and read_stream_begin_impl().

◆ GetPrivateRefCount()

static int32 GetPrivateRefCount ( Buffer  buffer)
inlinestatic

Definition at line 425 of file bufmgr.c.

426{
428
429 Assert(BufferIsValid(buffer));
430 Assert(!BufferIsLocal(buffer));
431
432 /*
433 * Not moving the entry - that's ok for the current users, but we might
434 * want to change this one day.
435 */
436 ref = GetPrivateRefCountEntry(buffer, false);
437
438 if (ref == NULL)
439 return 0;
440 return ref->refcount;
441}
static PrivateRefCountEntry * GetPrivateRefCountEntry(Buffer buffer, bool do_move)
Definition: bufmgr.c:351

References Assert(), PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsValid(), GetPrivateRefCountEntry(), and PrivateRefCountEntry::refcount.

Referenced by CheckBufferIsPinnedOnce(), ConditionalLockBufferForCleanup(), DebugPrintBufferRefcount(), HoldingBufferPinThatDelaysRecovery(), InvalidateBuffer(), InvalidateVictimBuffer(), IsBufferCleanupOK(), MarkBufferDirtyHint(), and ReadRecentBuffer().

◆ GetPrivateRefCountEntry()

static PrivateRefCountEntry * GetPrivateRefCountEntry ( Buffer  buffer,
bool  do_move 
)
static

Definition at line 351 of file bufmgr.c.

352{
354 int i;
355
356 Assert(BufferIsValid(buffer));
357 Assert(!BufferIsLocal(buffer));
358
359 /*
360 * First search for references in the array, that'll be sufficient in the
361 * majority of cases.
362 */
363 for (i = 0; i < REFCOUNT_ARRAY_ENTRIES; i++)
364 {
365 res = &PrivateRefCountArray[i];
366
367 if (res->buffer == buffer)
368 return res;
369 }
370
371 /*
372 * By here we know that the buffer, if already pinned, isn't residing in
373 * the array.
374 *
375 * Only look up the buffer in the hashtable if we've previously overflowed
376 * into it.
377 */
379 return NULL;
380
381 res = hash_search(PrivateRefCountHash, &buffer, HASH_FIND, NULL);
382
383 if (res == NULL)
384 return NULL;
385 else if (!do_move)
386 {
387 /* caller doesn't want us to move the hash entry into the array */
388 return res;
389 }
390 else
391 {
392 /* move buffer from hashtable into the free array slot */
393 bool found;
395
396 /* Ensure there's a free array slot */
398
399 /* Use up the reserved slot */
403 Assert(free->buffer == InvalidBuffer);
404
405 /* and fill it */
406 free->buffer = buffer;
407 free->refcount = res->refcount;
408
409 /* delete from hashtable */
411 Assert(found);
414
415 return free;
416 }
417}
#define free(a)
Definition: header.h:65
@ HASH_FIND
Definition: hsearch.h:113

References Assert(), PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsValid(), free, HASH_FIND, HASH_REMOVE, hash_search(), i, InvalidBuffer, PrivateRefCountArray, PrivateRefCountHash, PrivateRefCountOverflowed, PrivateRefCountEntry::refcount, REFCOUNT_ARRAY_ENTRIES, ReservedRefCountEntry, and ReservePrivateRefCountEntry().

Referenced by GetPrivateRefCount(), IncrBufferRefCount(), PinBuffer(), PinBuffer_Locked(), and UnpinBufferNoOwner().

◆ GetVictimBuffer()

static Buffer GetVictimBuffer ( BufferAccessStrategy  strategy,
IOContext  io_context 
)
static

Definition at line 2350 of file bufmgr.c.

2351{
2352 BufferDesc *buf_hdr;
2353 Buffer buf;
2354 uint32 buf_state;
2355 bool from_ring;
2356
2357 /*
2358 * Ensure, while the spinlock's not yet held, that there's a free refcount
2359 * entry, and a resource owner slot for the pin.
2360 */
2363
2364 /* we return here if a prospective victim buffer gets used concurrently */
2365again:
2366
2367 /*
2368 * Select a victim buffer. The buffer is returned with its header
2369 * spinlock still held!
2370 */
2371 buf_hdr = StrategyGetBuffer(strategy, &buf_state, &from_ring);
2372 buf = BufferDescriptorGetBuffer(buf_hdr);
2373
2374 Assert(BUF_STATE_GET_REFCOUNT(buf_state) == 0);
2375
2376 /* Pin the buffer and then release the buffer spinlock */
2377 PinBuffer_Locked(buf_hdr);
2378
2379 /*
2380 * We shouldn't have any other pins for this buffer.
2381 */
2383
2384 /*
2385 * If the buffer was dirty, try to write it out. There is a race
2386 * condition here, in that someone might dirty it after we released the
2387 * buffer header lock above, or even while we are writing it out (since
2388 * our share-lock won't prevent hint-bit updates). We will recheck the
2389 * dirty bit after re-locking the buffer header.
2390 */
2391 if (buf_state & BM_DIRTY)
2392 {
2393 LWLock *content_lock;
2394
2395 Assert(buf_state & BM_TAG_VALID);
2396 Assert(buf_state & BM_VALID);
2397
2398 /*
2399 * We need a share-lock on the buffer contents to write it out (else
2400 * we might write invalid data, eg because someone else is compacting
2401 * the page contents while we write). We must use a conditional lock
2402 * acquisition here to avoid deadlock. Even though the buffer was not
2403 * pinned (and therefore surely not locked) when StrategyGetBuffer
2404 * returned it, someone else could have pinned and exclusive-locked it
2405 * by the time we get here. If we try to get the lock unconditionally,
2406 * we'd block waiting for them; if they later block waiting for us,
2407 * deadlock ensues. (This has been observed to happen when two
2408 * backends are both trying to split btree index pages, and the second
2409 * one just happens to be trying to split the page the first one got
2410 * from StrategyGetBuffer.)
2411 */
2412 content_lock = BufferDescriptorGetContentLock(buf_hdr);
2413 if (!LWLockConditionalAcquire(content_lock, LW_SHARED))
2414 {
2415 /*
2416 * Someone else has locked the buffer, so give it up and loop back
2417 * to get another one.
2418 */
2419 UnpinBuffer(buf_hdr);
2420 goto again;
2421 }
2422
2423 /*
2424 * If using a nondefault strategy, and writing the buffer would
2425 * require a WAL flush, let the strategy decide whether to go ahead
2426 * and write/reuse the buffer or to choose another victim. We need a
2427 * lock to inspect the page LSN, so this can't be done inside
2428 * StrategyGetBuffer.
2429 */
2430 if (strategy != NULL)
2431 {
2432 XLogRecPtr lsn;
2433
2434 /* Read the LSN while holding buffer header lock */
2435 buf_state = LockBufHdr(buf_hdr);
2436 lsn = BufferGetLSN(buf_hdr);
2437 UnlockBufHdr(buf_hdr, buf_state);
2438
2439 if (XLogNeedsFlush(lsn)
2440 && StrategyRejectBuffer(strategy, buf_hdr, from_ring))
2441 {
2442 LWLockRelease(content_lock);
2443 UnpinBuffer(buf_hdr);
2444 goto again;
2445 }
2446 }
2447
2448 /* OK, do the I/O */
2449 FlushBuffer(buf_hdr, NULL, IOOBJECT_RELATION, io_context);
2450 LWLockRelease(content_lock);
2451
2453 &buf_hdr->tag);
2454 }
2455
2456
2457 if (buf_state & BM_VALID)
2458 {
2459 /*
2460 * When a BufferAccessStrategy is in use, blocks evicted from shared
2461 * buffers are counted as IOOP_EVICT in the corresponding context
2462 * (e.g. IOCONTEXT_BULKWRITE). Shared buffers are evicted by a
2463 * strategy in two cases: 1) while initially claiming buffers for the
2464 * strategy ring 2) to replace an existing strategy ring buffer
2465 * because it is pinned or in use and cannot be reused.
2466 *
2467 * Blocks evicted from buffers already in the strategy ring are
2468 * counted as IOOP_REUSE in the corresponding strategy context.
2469 *
2470 * At this point, we can accurately count evictions and reuses,
2471 * because we have successfully claimed the valid buffer. Previously,
2472 * we may have been forced to release the buffer due to concurrent
2473 * pinners or erroring out.
2474 */
2476 from_ring ? IOOP_REUSE : IOOP_EVICT, 1, 0);
2477 }
2478
2479 /*
2480 * If the buffer has an entry in the buffer mapping table, delete it. This
2481 * can fail because another backend could have pinned or dirtied the
2482 * buffer.
2483 */
2484 if ((buf_state & BM_TAG_VALID) && !InvalidateVictimBuffer(buf_hdr))
2485 {
2486 UnpinBuffer(buf_hdr);
2487 goto again;
2488 }
2489
2490 /* a final set of sanity checks */
2491#ifdef USE_ASSERT_CHECKING
2492 buf_state = pg_atomic_read_u32(&buf_hdr->state);
2493
2494 Assert(BUF_STATE_GET_REFCOUNT(buf_state) == 1);
2495 Assert(!(buf_state & (BM_TAG_VALID | BM_VALID | BM_DIRTY)));
2496
2498#endif
2499
2500 return buf;
2501}
WritebackContext BackendWritebackContext
Definition: buf_init.c:24
void CheckBufferIsPinnedOnce(Buffer buffer)
Definition: bufmgr.c:5652
void ScheduleBufferTagForWriteback(WritebackContext *wb_context, IOContext io_context, BufferTag *tag)
Definition: bufmgr.c:6411
BufferDesc * StrategyGetBuffer(BufferAccessStrategy strategy, uint32 *buf_state, bool *from_ring)
Definition: freelist.c:196
bool StrategyRejectBuffer(BufferAccessStrategy strategy, BufferDesc *buf, bool from_ring)
Definition: freelist.c:840
@ IOOP_EVICT
Definition: pgstat.h:304
@ IOOP_REUSE
Definition: pgstat.h:307
bool XLogNeedsFlush(XLogRecPtr record)
Definition: xlog.c:3254

References Assert(), BackendWritebackContext, BM_DIRTY, BM_TAG_VALID, BM_VALID, buf, BUF_STATE_GET_REFCOUNT, BufferDescriptorGetBuffer(), BufferDescriptorGetContentLock(), BufferGetLSN, CheckBufferIsPinnedOnce(), CurrentResourceOwner, FlushBuffer(), InvalidateVictimBuffer(), IOOBJECT_RELATION, IOOP_EVICT, IOOP_REUSE, LockBufHdr(), LW_SHARED, LWLockConditionalAcquire(), LWLockRelease(), pg_atomic_read_u32(), pgstat_count_io_op(), PinBuffer_Locked(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), ScheduleBufferTagForWriteback(), BufferDesc::state, StrategyGetBuffer(), StrategyRejectBuffer(), BufferDesc::tag, UnlockBufHdr(), UnpinBuffer(), and XLogNeedsFlush().

Referenced by BufferAlloc(), and ExtendBufferedRelShared().

◆ HoldingBufferPinThatDelaysRecovery()

bool HoldingBufferPinThatDelaysRecovery ( void  )

Definition at line 5827 of file bufmgr.c.

5828{
5829 int bufid = GetStartupBufferPinWaitBufId();
5830
5831 /*
5832 * If we get woken slowly then it's possible that the Startup process was
5833 * already woken by other backends before we got here. Also possible that
5834 * we get here by multiple interrupts or interrupts at inappropriate
5835 * times, so make sure we do nothing if the bufid is not set.
5836 */
5837 if (bufid < 0)
5838 return false;
5839
5840 if (GetPrivateRefCount(bufid + 1) > 0)
5841 return true;
5842
5843 return false;
5844}
int GetStartupBufferPinWaitBufId(void)
Definition: proc.c:767

References GetPrivateRefCount(), and GetStartupBufferPinWaitBufId().

Referenced by CheckRecoveryConflictDeadlock(), and ProcessRecoveryConflictInterrupt().

◆ IncrBufferRefCount()

void IncrBufferRefCount ( Buffer  buffer)

◆ InitBufferManagerAccess()

void InitBufferManagerAccess ( void  )

Definition at line 4013 of file bufmgr.c.

4014{
4015 HASHCTL hash_ctl;
4016
4017 /*
4018 * An advisory limit on the number of pins each backend should hold, based
4019 * on shared_buffers and the maximum number of connections possible.
4020 * That's very pessimistic, but outside toy-sized shared_buffers it should
4021 * allow plenty of pins. LimitAdditionalPins() and
4022 * GetAdditionalPinLimit() can be used to check the remaining balance.
4023 */
4025
4026 memset(&PrivateRefCountArray, 0, sizeof(PrivateRefCountArray));
4027
4028 hash_ctl.keysize = sizeof(int32);
4029 hash_ctl.entrysize = sizeof(PrivateRefCountEntry);
4030
4031 PrivateRefCountHash = hash_create("PrivateRefCount", 100, &hash_ctl,
4033
4034 /*
4035 * AtProcExit_Buffers needs LWLock access, and thereby has to be called at
4036 * the corresponding phase of backend shutdown.
4037 */
4038 Assert(MyProc != NULL);
4040}
static void AtProcExit_Buffers(int code, Datum arg)
Definition: bufmgr.c:4047
struct PrivateRefCountEntry PrivateRefCountEntry
HTAB * hash_create(const char *tabname, long nelem, const HASHCTL *info, int flags)
Definition: dynahash.c:352
int MaxBackends
Definition: globals.c:147
#define HASH_ELEM
Definition: hsearch.h:95
#define HASH_BLOBS
Definition: hsearch.h:97
void on_shmem_exit(pg_on_exit_callback function, Datum arg)
Definition: ipc.c:365
#define NUM_AUXILIARY_PROCS
Definition: proc.h:455
PGPROC * MyProc
Definition: proc.c:67
Size keysize
Definition: hsearch.h:75
Size entrysize
Definition: hsearch.h:76

References Assert(), AtProcExit_Buffers(), HASHCTL::entrysize, HASH_BLOBS, hash_create(), HASH_ELEM, HASHCTL::keysize, MaxBackends, MaxProportionalPins, MyProc, NBuffers, NUM_AUXILIARY_PROCS, on_shmem_exit(), PrivateRefCountArray, and PrivateRefCountHash.

Referenced by BaseInit().

◆ InvalidateBuffer()

static void InvalidateBuffer ( BufferDesc buf)
static

Definition at line 2183 of file bufmgr.c.

2184{
2185 BufferTag oldTag;
2186 uint32 oldHash; /* hash value for oldTag */
2187 LWLock *oldPartitionLock; /* buffer partition lock for it */
2188 uint32 oldFlags;
2189 uint32 buf_state;
2190
2191 /* Save the original buffer tag before dropping the spinlock */
2192 oldTag = buf->tag;
2193
2194 buf_state = pg_atomic_read_u32(&buf->state);
2195 Assert(buf_state & BM_LOCKED);
2196 UnlockBufHdr(buf, buf_state);
2197
2198 /*
2199 * Need to compute the old tag's hashcode and partition lock ID. XXX is it
2200 * worth storing the hashcode in BufferDesc so we need not recompute it
2201 * here? Probably not.
2202 */
2203 oldHash = BufTableHashCode(&oldTag);
2204 oldPartitionLock = BufMappingPartitionLock(oldHash);
2205
2206retry:
2207
2208 /*
2209 * Acquire exclusive mapping lock in preparation for changing the buffer's
2210 * association.
2211 */
2212 LWLockAcquire(oldPartitionLock, LW_EXCLUSIVE);
2213
2214 /* Re-lock the buffer header */
2215 buf_state = LockBufHdr(buf);
2216
2217 /* If it's changed while we were waiting for lock, do nothing */
2218 if (!BufferTagsEqual(&buf->tag, &oldTag))
2219 {
2220 UnlockBufHdr(buf, buf_state);
2221 LWLockRelease(oldPartitionLock);
2222 return;
2223 }
2224
2225 /*
2226 * We assume the reason for it to be pinned is that either we were
2227 * asynchronously reading the page in before erroring out or someone else
2228 * is flushing the page out. Wait for the IO to finish. (This could be
2229 * an infinite loop if the refcount is messed up... it would be nice to
2230 * time out after awhile, but there seems no way to be sure how many loops
2231 * may be needed. Note that if the other guy has pinned the buffer but
2232 * not yet done StartBufferIO, WaitIO will fall through and we'll
2233 * effectively be busy-looping here.)
2234 */
2235 if (BUF_STATE_GET_REFCOUNT(buf_state) != 0)
2236 {
2237 UnlockBufHdr(buf, buf_state);
2238 LWLockRelease(oldPartitionLock);
2239 /* safety check: should definitely not be our *own* pin */
2241 elog(ERROR, "buffer is pinned in InvalidateBuffer");
2242 WaitIO(buf);
2243 goto retry;
2244 }
2245
2246 /*
2247 * Clear out the buffer's tag and flags. We must do this to ensure that
2248 * linear scans of the buffer array don't think the buffer is valid.
2249 */
2250 oldFlags = buf_state & BUF_FLAG_MASK;
2251 ClearBufferTag(&buf->tag);
2252 buf_state &= ~(BUF_FLAG_MASK | BUF_USAGECOUNT_MASK);
2253 UnlockBufHdr(buf, buf_state);
2254
2255 /*
2256 * Remove the buffer from the lookup hashtable, if it was in there.
2257 */
2258 if (oldFlags & BM_TAG_VALID)
2259 BufTableDelete(&oldTag, oldHash);
2260
2261 /*
2262 * Done with mapping lock.
2263 */
2264 LWLockRelease(oldPartitionLock);
2265
2266 /*
2267 * Insert the buffer at the head of the list of free buffers.
2268 */
2270}
#define BUF_USAGECOUNT_MASK
Definition: buf_internals.h:53
static bool BufferTagsEqual(const BufferTag *tag1, const BufferTag *tag2)
static void ClearBufferTag(BufferTag *tag)
void BufTableDelete(BufferTag *tagPtr, uint32 hashcode)
Definition: buf_table.c:148
static void WaitIO(BufferDesc *buf)
Definition: bufmgr.c:5964

References Assert(), BM_LOCKED, BM_TAG_VALID, buf, BUF_FLAG_MASK, BUF_STATE_GET_REFCOUNT, BUF_USAGECOUNT_MASK, BufferDescriptorGetBuffer(), BufferTagsEqual(), BufMappingPartitionLock(), BufTableDelete(), BufTableHashCode(), ClearBufferTag(), elog, ERROR, GetPrivateRefCount(), LockBufHdr(), LW_EXCLUSIVE, LWLockAcquire(), LWLockRelease(), pg_atomic_read_u32(), StrategyFreeBuffer(), UnlockBufHdr(), and WaitIO().

Referenced by DropDatabaseBuffers(), DropRelationBuffers(), DropRelationsAllBuffers(), and FindAndDropRelationBuffers().

◆ InvalidateVictimBuffer()

static bool InvalidateVictimBuffer ( BufferDesc buf_hdr)
static

Definition at line 2282 of file bufmgr.c.

2283{
2284 uint32 buf_state;
2285 uint32 hash;
2286 LWLock *partition_lock;
2287 BufferTag tag;
2288
2290
2291 /* have buffer pinned, so it's safe to read tag without lock */
2292 tag = buf_hdr->tag;
2293
2294 hash = BufTableHashCode(&tag);
2295 partition_lock = BufMappingPartitionLock(hash);
2296
2297 LWLockAcquire(partition_lock, LW_EXCLUSIVE);
2298
2299 /* lock the buffer header */
2300 buf_state = LockBufHdr(buf_hdr);
2301
2302 /*
2303 * We have the buffer pinned nobody else should have been able to unset
2304 * this concurrently.
2305 */
2306 Assert(buf_state & BM_TAG_VALID);
2307 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
2308 Assert(BufferTagsEqual(&buf_hdr->tag, &tag));
2309
2310 /*
2311 * If somebody else pinned the buffer since, or even worse, dirtied it,
2312 * give up on this buffer: It's clearly in use.
2313 */
2314 if (BUF_STATE_GET_REFCOUNT(buf_state) != 1 || (buf_state & BM_DIRTY))
2315 {
2316 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
2317
2318 UnlockBufHdr(buf_hdr, buf_state);
2319 LWLockRelease(partition_lock);
2320
2321 return false;
2322 }
2323
2324 /*
2325 * Clear out the buffer's tag and flags and usagecount. This is not
2326 * strictly required, as BM_TAG_VALID/BM_VALID needs to be checked before
2327 * doing anything with the buffer. But currently it's beneficial, as the
2328 * cheaper pre-check for several linear scans of shared buffers use the
2329 * tag (see e.g. FlushDatabaseBuffers()).
2330 */
2331 ClearBufferTag(&buf_hdr->tag);
2332 buf_state &= ~(BUF_FLAG_MASK | BUF_USAGECOUNT_MASK);
2333 UnlockBufHdr(buf_hdr, buf_state);
2334
2335 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
2336
2337 /* finally delete buffer from the buffer mapping table */
2338 BufTableDelete(&tag, hash);
2339
2340 LWLockRelease(partition_lock);
2341
2342 Assert(!(buf_state & (BM_DIRTY | BM_VALID | BM_TAG_VALID)));
2343 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
2345
2346 return true;
2347}

References Assert(), BM_DIRTY, BM_TAG_VALID, BM_VALID, BUF_FLAG_MASK, BUF_STATE_GET_REFCOUNT, BUF_USAGECOUNT_MASK, BufferDescriptorGetBuffer(), BufferTagsEqual(), BufMappingPartitionLock(), BufTableDelete(), BufTableHashCode(), ClearBufferTag(), GetPrivateRefCount(), hash(), LockBufHdr(), LW_EXCLUSIVE, LWLockAcquire(), LWLockRelease(), pg_atomic_read_u32(), BufferDesc::state, BufferDesc::tag, and UnlockBufHdr().

Referenced by EvictUnpinnedBufferInternal(), and GetVictimBuffer().

◆ IsBufferCleanupOK()

bool IsBufferCleanupOK ( Buffer  buffer)

Definition at line 5911 of file bufmgr.c.

5912{
5913 BufferDesc *bufHdr;
5914 uint32 buf_state;
5915
5916 Assert(BufferIsValid(buffer));
5917
5918 /* see AIO related comment in LockBufferForCleanup() */
5919
5920 if (BufferIsLocal(buffer))
5921 {
5922 /* There should be exactly one pin */
5923 if (LocalRefCount[-buffer - 1] != 1)
5924 return false;
5925 /* Nobody else to wait for */
5926 return true;
5927 }
5928
5929 /* There should be exactly one local pin */
5930 if (GetPrivateRefCount(buffer) != 1)
5931 return false;
5932
5933 bufHdr = GetBufferDescriptor(buffer - 1);
5934
5935 /* caller must hold exclusive lock on buffer */
5937 LW_EXCLUSIVE));
5938
5939 buf_state = LockBufHdr(bufHdr);
5940
5941 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
5942 if (BUF_STATE_GET_REFCOUNT(buf_state) == 1)
5943 {
5944 /* pincount is OK. */
5945 UnlockBufHdr(bufHdr, buf_state);
5946 return true;
5947 }
5948
5949 UnlockBufHdr(bufHdr, buf_state);
5950 return false;
5951}

References Assert(), BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsValid(), GetBufferDescriptor(), GetPrivateRefCount(), LocalRefCount, LockBufHdr(), LW_EXCLUSIVE, LWLockHeldByMeInMode(), and UnlockBufHdr().

Referenced by _hash_doinsert(), _hash_expandtable(), _hash_splitbucket(), and hashbucketcleanup().

◆ IssuePendingWritebacks()

void IssuePendingWritebacks ( WritebackContext wb_context,
IOContext  io_context 
)

Definition at line 6461 of file bufmgr.c.

6462{
6463 instr_time io_start;
6464 int i;
6465
6466 if (wb_context->nr_pending == 0)
6467 return;
6468
6469 /*
6470 * Executing the writes in-order can make them a lot faster, and allows to
6471 * merge writeback requests to consecutive blocks into larger writebacks.
6472 */
6473 sort_pending_writebacks(wb_context->pending_writebacks,
6474 wb_context->nr_pending);
6475
6477
6478 /*
6479 * Coalesce neighbouring writes, but nothing else. For that we iterate
6480 * through the, now sorted, array of pending flushes, and look forward to
6481 * find all neighbouring (or identical) writes.
6482 */
6483 for (i = 0; i < wb_context->nr_pending; i++)
6484 {
6487 SMgrRelation reln;
6488 int ahead;
6489 BufferTag tag;
6490 RelFileLocator currlocator;
6491 Size nblocks = 1;
6492
6493 cur = &wb_context->pending_writebacks[i];
6494 tag = cur->tag;
6495 currlocator = BufTagGetRelFileLocator(&tag);
6496
6497 /*
6498 * Peek ahead, into following writeback requests, to see if they can
6499 * be combined with the current one.
6500 */
6501 for (ahead = 0; i + ahead + 1 < wb_context->nr_pending; ahead++)
6502 {
6503
6504 next = &wb_context->pending_writebacks[i + ahead + 1];
6505
6506 /* different file, stop */
6507 if (!RelFileLocatorEquals(currlocator,
6508 BufTagGetRelFileLocator(&next->tag)) ||
6509 BufTagGetForkNum(&cur->tag) != BufTagGetForkNum(&next->tag))
6510 break;
6511
6512 /* ok, block queued twice, skip */
6513 if (cur->tag.blockNum == next->tag.blockNum)
6514 continue;
6515
6516 /* only merge consecutive writes */
6517 if (cur->tag.blockNum + 1 != next->tag.blockNum)
6518 break;
6519
6520 nblocks++;
6521 cur = next;
6522 }
6523
6524 i += ahead;
6525
6526 /* and finally tell the kernel to write the data to storage */
6527 reln = smgropen(currlocator, INVALID_PROC_NUMBER);
6528 smgrwriteback(reln, BufTagGetForkNum(&tag), tag.blockNum, nblocks);
6529 }
6530
6531 /*
6532 * Assume that writeback requests are only issued for buffers containing
6533 * blocks of permanent relations.
6534 */
6536 IOOP_WRITEBACK, io_start, wb_context->nr_pending, 0);
6537
6538 wb_context->nr_pending = 0;
6539}
static int32 next
Definition: blutils.c:224
struct cursor * cur
Definition: ecpg.c:29
@ IOOP_WRITEBACK
Definition: pgstat.h:308
#define RelFileLocatorEquals(locator1, locator2)
void smgrwriteback(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, BlockNumber nblocks)
Definition: smgr.c:805
PendingWriteback pending_writebacks[WRITEBACK_MAX_PENDING_FLUSHES]

References buftag::blockNum, BufTagGetForkNum(), BufTagGetRelFileLocator(), cur, i, INVALID_PROC_NUMBER, IOOBJECT_RELATION, IOOP_WRITEBACK, next, WritebackContext::nr_pending, WritebackContext::pending_writebacks, pgstat_count_io_op_time(), pgstat_prepare_io_time(), RelFileLocatorEquals, smgropen(), smgrwriteback(), and track_io_timing.

Referenced by BufferSync(), and ScheduleBufferTagForWriteback().

◆ LimitAdditionalPins()

void LimitAdditionalPins ( uint32 additional_pins)

Definition at line 2548 of file bufmgr.c.

2549{
2550 uint32 limit;
2551
2552 if (*additional_pins <= 1)
2553 return;
2554
2555 limit = GetAdditionalPinLimit();
2556 limit = Max(limit, 1);
2557 if (limit < *additional_pins)
2558 *additional_pins = limit;
2559}
uint32 GetAdditionalPinLimit(void)
Definition: bufmgr.c:2522
#define Max(x, y)
Definition: c.h:969

References GetAdditionalPinLimit(), and Max.

Referenced by ExtendBufferedRelShared().

◆ local_buffer_readv_complete()

static PgAioResult local_buffer_readv_complete ( PgAioHandle ioh,
PgAioResult  prior_result,
uint8  cb_data 
)
static

Definition at line 7428 of file bufmgr.c.

7430{
7431 return buffer_readv_complete(ioh, prior_result, cb_data, true);
7432}
static pg_attribute_always_inline PgAioResult buffer_readv_complete(PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data, bool is_temp)
Definition: bufmgr.c:7175

References buffer_readv_complete().

◆ local_buffer_readv_stage()

static void local_buffer_readv_stage ( PgAioHandle ioh,
uint8  cb_data 
)
static

Definition at line 7422 of file bufmgr.c.

7423{
7424 buffer_stage_common(ioh, false, true);
7425}
static pg_attribute_always_inline void buffer_stage_common(PgAioHandle *ioh, bool is_write, bool is_temp)
Definition: bufmgr.c:6785

References buffer_stage_common().

◆ local_buffer_write_error_callback()

static void local_buffer_write_error_callback ( void *  arg)
static

Definition at line 6214 of file bufmgr.c.

6215{
6216 BufferDesc *bufHdr = (BufferDesc *) arg;
6217
6218 if (bufHdr != NULL)
6219 errcontext("writing block %u of relation %s",
6220 bufHdr->tag.blockNum,
6223 BufTagGetForkNum(&bufHdr->tag)).str);
6224}
#define errcontext
Definition: elog.h:197
void * arg

References arg, buftag::blockNum, BufTagGetForkNum(), BufTagGetRelFileLocator(), errcontext, MyProcNumber, relpathbackend, and BufferDesc::tag.

Referenced by FlushRelationBuffers().

◆ LockBuffer()

void LockBuffer ( Buffer  buffer,
int  mode 
)

Definition at line 5605 of file bufmgr.c.

5606{
5607 BufferDesc *buf;
5608
5609 Assert(BufferIsPinned(buffer));
5610 if (BufferIsLocal(buffer))
5611 return; /* local buffers need no lock */
5612
5613 buf = GetBufferDescriptor(buffer - 1);
5614
5615 if (mode == BUFFER_LOCK_UNLOCK)
5617 else if (mode == BUFFER_LOCK_SHARE)
5619 else if (mode == BUFFER_LOCK_EXCLUSIVE)
5621 else
5622 elog(ERROR, "unrecognized buffer lock mode: %d", mode);
5623}
#define BUFFER_LOCK_SHARE
Definition: bufmgr.h:197
#define BUFFER_LOCK_EXCLUSIVE
Definition: bufmgr.h:198

References Assert(), buf, PrivateRefCountEntry::buffer, BUFFER_LOCK_EXCLUSIVE, BUFFER_LOCK_SHARE, BUFFER_LOCK_UNLOCK, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsPinned, elog, ERROR, GetBufferDescriptor(), LW_EXCLUSIVE, LW_SHARED, LWLockAcquire(), LWLockRelease(), and mode.

Referenced by _bt_lockbuf(), _bt_unlockbuf(), _bt_upgradelockbufcleanup(), _hash_addovflpage(), _hash_doinsert(), _hash_expandtable(), _hash_finish_split(), _hash_first(), _hash_freeovflpage(), _hash_getbuf(), _hash_getbuf_with_strategy(), _hash_getcachedmetap(), _hash_init(), _hash_kill_items(), _hash_readnext(), _hash_readpage(), _hash_readprev(), _hash_splitbucket(), _hash_squeezebucket(), _hash_vacuum_one_page(), BitmapHeapScanNextBlock(), blbulkdelete(), blgetbitmap(), blinsert(), BloomInitMetapage(), BloomNewBuffer(), blvacuumcleanup(), brin_doinsert(), brin_doupdate(), brin_evacuate_page(), brin_getinsertbuffer(), brin_page_cleanup(), bringetbitmap(), brinGetStats(), brinGetTupleForHeapBlock(), brininsert(), brinLockRevmapPageForUpdate(), brinRevmapDesummarizeRange(), brinRevmapInitialize(), brinsummarize(), bt_metap(), bt_multi_page_stats(), bt_page_items_internal(), bt_page_stats_internal(), bt_recheck_sibling_links(), collect_corrupt_items(), collect_visibility_data(), collectMatchBitmap(), ConditionalLockBufferForCleanup(), count_nondeletable_pages(), create_toy_buffer(), entryLoadMoreItems(), FreeSpaceMapPrepareTruncateRel(), fsm_readbuf(), fsm_search(), fsm_search_avail(), fsm_set_and_search(), fsm_vacuum_page(), get_raw_page_internal(), GetVisibilityMapPins(), gin_check_parent_keys_consistency(), gin_check_posting_tree_parent_keys_consistency(), gin_refind_parent(), ginbulkdelete(), ginEntryInsert(), ginFindLeafPage(), ginFindParents(), ginFinishOldSplit(), ginFinishSplit(), ginGetStats(), ginHeapTupleFastInsert(), ginInsertCleanup(), ginInsertValue(), GinNewBuffer(), ginScanToDelete(), ginStepRight(), ginTraverseLock(), ginUpdateStats(), ginvacuumcleanup(), ginVacuumPostingTreeLeaves(), gistBufferingFindCorrectParent(), gistbufferinginserttuples(), gistdoinsert(), gistFindCorrectParent(), gistFindPath(), gistfinishsplit(), gistfixsplit(), gistformdownlink(), gistGetMaxLevel(), gistinserttuples(), gistkillitems(), gistNewBuffer(), gistProcessItup(), gistScanPage(), gistvacuum_delete_empty_pages(), gistvacuumpage(), hashbucketcleanup(), hashbulkdelete(), heap_abort_speculative(), heap_delete(), heap_fetch(), heap_finish_speculative(), heap_get_latest_tid(), heap_index_delete_tuples(), heap_inplace_lock(), heap_inplace_unlock(), heap_inplace_update_and_unlock(), heap_lock_tuple(), heap_lock_updated_tuple_rec(), heap_page_prune_opt(), heap_prepare_pagescan(), heap_update(), heap_xlog_visible(), heapam_index_build_range_scan(), heapam_index_fetch_tuple(), heapam_index_validate_scan(), heapam_relation_copy_for_cluster(), heapam_scan_analyze_next_block(), heapam_scan_sample_next_tuple(), heapam_tuple_satisfies_snapshot(), heapgettup(), initBloomState(), invalidate_rel_block(), lazy_scan_heap(), lazy_scan_new_or_empty(), lazy_vacuum_heap_rel(), LockBufferForCleanup(), log_newpage_range(), modify_rel_block(), palloc_btree_page(), pg_visibility(), pgrowlocks(), pgstat_btree_page(), pgstat_gist_page(), pgstat_heap(), pgstatginindex_internal(), pgstathashindex(), pgstatindex_impl(), read_seq_tuple(), RelationAddBlocks(), RelationCopyStorageUsingBuffer(), RelationGetBufferForTuple(), revmap_physical_extend(), scanGetCandidate(), scanPendingInsert(), ScanSourceDatabasePgClass(), shiftList(), spgdoinsert(), spgGetCache(), SpGistNewBuffer(), spgprocesspending(), spgvacuumpage(), spgWalk(), startScanEntry(), statapprox_heap(), summarize_range(), UnlockReleaseBuffer(), verify_heapam(), verifyBackupPageConsistency(), visibilitymap_clear(), visibilitymap_prepare_truncate(), visibilitymap_set(), vm_readbuf(), XLogReadBufferForRedoExtended(), XLogRecordPageWithFreeSpace(), and ZeroAndLockBuffer().

◆ LockBufferForCleanup()

void LockBufferForCleanup ( Buffer  buffer)

Definition at line 5685 of file bufmgr.c.

5686{
5687 BufferDesc *bufHdr;
5688 TimestampTz waitStart = 0;
5689 bool waiting = false;
5690 bool logged_recovery_conflict = false;
5691
5692 Assert(BufferIsPinned(buffer));
5693 Assert(PinCountWaitBuf == NULL);
5694
5696
5697 /*
5698 * We do not yet need to be worried about in-progress AIOs holding a pin,
5699 * as we, so far, only support doing reads via AIO and this function can
5700 * only be called once the buffer is valid (i.e. no read can be in
5701 * flight).
5702 */
5703
5704 /* Nobody else to wait for */
5705 if (BufferIsLocal(buffer))
5706 return;
5707
5708 bufHdr = GetBufferDescriptor(buffer - 1);
5709
5710 for (;;)
5711 {
5712 uint32 buf_state;
5713
5714 /* Try to acquire lock */
5716 buf_state = LockBufHdr(bufHdr);
5717
5718 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
5719 if (BUF_STATE_GET_REFCOUNT(buf_state) == 1)
5720 {
5721 /* Successfully acquired exclusive lock with pincount 1 */
5722 UnlockBufHdr(bufHdr, buf_state);
5723
5724 /*
5725 * Emit the log message if recovery conflict on buffer pin was
5726 * resolved but the startup process waited longer than
5727 * deadlock_timeout for it.
5728 */
5729 if (logged_recovery_conflict)
5731 waitStart, GetCurrentTimestamp(),
5732 NULL, false);
5733
5734 if (waiting)
5735 {
5736 /* reset ps display to remove the suffix if we added one */
5738 waiting = false;
5739 }
5740 return;
5741 }
5742 /* Failed, so mark myself as waiting for pincount 1 */
5743 if (buf_state & BM_PIN_COUNT_WAITER)
5744 {
5745 UnlockBufHdr(bufHdr, buf_state);
5747 elog(ERROR, "multiple backends attempting to wait for pincount 1");
5748 }
5750 PinCountWaitBuf = bufHdr;
5751 buf_state |= BM_PIN_COUNT_WAITER;
5752 UnlockBufHdr(bufHdr, buf_state);
5754
5755 /* Wait to be signaled by UnpinBuffer() */
5756 if (InHotStandby)
5757 {
5758 if (!waiting)
5759 {
5760 /* adjust the process title to indicate that it's waiting */
5761 set_ps_display_suffix("waiting");
5762 waiting = true;
5763 }
5764
5765 /*
5766 * Emit the log message if the startup process is waiting longer
5767 * than deadlock_timeout for recovery conflict on buffer pin.
5768 *
5769 * Skip this if first time through because the startup process has
5770 * not started waiting yet in this case. So, the wait start
5771 * timestamp is set after this logic.
5772 */
5773 if (waitStart != 0 && !logged_recovery_conflict)
5774 {
5776
5777 if (TimestampDifferenceExceeds(waitStart, now,
5779 {
5781 waitStart, now, NULL, true);
5782 logged_recovery_conflict = true;
5783 }
5784 }
5785
5786 /*
5787 * Set the wait start timestamp if logging is enabled and first
5788 * time through.
5789 */
5790 if (log_recovery_conflict_waits && waitStart == 0)
5791 waitStart = GetCurrentTimestamp();
5792
5793 /* Publish the bufid that Startup process waits on */
5794 SetStartupBufferPinWaitBufId(buffer - 1);
5795 /* Set alarm and then wait to be signaled by UnpinBuffer() */
5797 /* Reset the published bufid */
5799 }
5800 else
5801 ProcWaitForSignal(WAIT_EVENT_BUFFER_PIN);
5802
5803 /*
5804 * Remove flag marking us as waiter. Normally this will not be set
5805 * anymore, but ProcWaitForSignal() can return for other signals as
5806 * well. We take care to only reset the flag if we're the waiter, as
5807 * theoretically another backend could have started waiting. That's
5808 * impossible with the current usages due to table level locking, but
5809 * better be safe.
5810 */
5811 buf_state = LockBufHdr(bufHdr);
5812 if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
5814 buf_state &= ~BM_PIN_COUNT_WAITER;
5815 UnlockBufHdr(bufHdr, buf_state);
5816
5817 PinCountWaitBuf = NULL;
5818 /* Loop back and try again */
5819 }
5820}
bool TimestampDifferenceExceeds(TimestampTz start_time, TimestampTz stop_time, int msec)
Definition: timestamp.c:1781
TimestampTz GetCurrentTimestamp(void)
Definition: timestamp.c:1645
Datum now(PG_FUNCTION_ARGS)
Definition: timestamp.c:1609
#define BM_PIN_COUNT_WAITER
Definition: buf_internals.h:75
static BufferDesc * PinCountWaitBuf
Definition: bufmgr.c:183
int64 TimestampTz
Definition: timestamp.h:39
@ PROCSIG_RECOVERY_CONFLICT_BUFFERPIN
Definition: procsignal.h:48
void set_ps_display_remove_suffix(void)
Definition: ps_status.c:423
void set_ps_display_suffix(const char *suffix)
Definition: ps_status.c:371
int DeadlockTimeout
Definition: proc.c:58
void SetStartupBufferPinWaitBufId(int bufid)
Definition: proc.c:755
void ProcWaitForSignal(uint32 wait_event_info)
Definition: proc.c:1975
void ResolveRecoveryConflictWithBufferPin(void)
Definition: standby.c:793
bool log_recovery_conflict_waits
Definition: standby.c:42
void LogRecoveryConflict(ProcSignalReason reason, TimestampTz wait_start, TimestampTz now, VirtualTransactionId *wait_list, bool still_waiting)
Definition: standby.c:274
int wait_backend_pgprocno
static volatile sig_atomic_t waiting
Definition: waiteventset.c:170
#define InHotStandby
Definition: xlogutils.h:60

References Assert(), BM_PIN_COUNT_WAITER, BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BUFFER_LOCK_EXCLUSIVE, BUFFER_LOCK_UNLOCK, BufferIsLocal, BufferIsPinned, CheckBufferIsPinnedOnce(), DeadlockTimeout, elog, ERROR, GetBufferDescriptor(), GetCurrentTimestamp(), InHotStandby, LockBuffer(), LockBufHdr(), log_recovery_conflict_waits, LogRecoveryConflict(), MyProcNumber, now(), PinCountWaitBuf, PROCSIG_RECOVERY_CONFLICT_BUFFERPIN, ProcWaitForSignal(), ResolveRecoveryConflictWithBufferPin(), set_ps_display_remove_suffix(), set_ps_display_suffix(), SetStartupBufferPinWaitBufId(), TimestampDifferenceExceeds(), UnlockBufHdr(), BufferDesc::wait_backend_pgprocno, and waiting.

Referenced by _bt_upgradelockbufcleanup(), ginVacuumPostingTree(), hashbulkdelete(), heap_force_common(), lazy_scan_heap(), XLogReadBufferForRedoExtended(), and ZeroAndLockBuffer().

◆ LockBufHdr()

uint32 LockBufHdr ( BufferDesc desc)

Definition at line 6257 of file bufmgr.c.

6258{
6259 SpinDelayStatus delayStatus;
6260 uint32 old_buf_state;
6261
6263
6264 init_local_spin_delay(&delayStatus);
6265
6266 while (true)
6267 {
6268 /* set BM_LOCKED flag */
6269 old_buf_state = pg_atomic_fetch_or_u32(&desc->state, BM_LOCKED);
6270 /* if it wasn't set before we're OK */
6271 if (!(old_buf_state & BM_LOCKED))
6272 break;
6273 perform_spin_delay(&delayStatus);
6274 }
6275 finish_spin_delay(&delayStatus);
6276 return old_buf_state | BM_LOCKED;
6277}
static uint32 pg_atomic_fetch_or_u32(volatile pg_atomic_uint32 *ptr, uint32 or_)
Definition: atomics.h:410
void perform_spin_delay(SpinDelayStatus *status)
Definition: s_lock.c:126
void finish_spin_delay(SpinDelayStatus *status)
Definition: s_lock.c:186
#define init_local_spin_delay(status)
Definition: s_lock.h:751

References Assert(), BM_LOCKED, BufferDescriptorGetBuffer(), BufferIsLocal, finish_spin_delay(), init_local_spin_delay, perform_spin_delay(), pg_atomic_fetch_or_u32(), and BufferDesc::state.

Referenced by AbortBufferIO(), apw_dump_now(), buffer_stage_common(), BufferAlloc(), BufferGetLSNAtomic(), BufferSync(), ConditionalLockBufferForCleanup(), create_toy_buffer(), DropDatabaseBuffers(), DropRelationBuffers(), DropRelationsAllBuffers(), EvictAllUnpinnedBuffers(), EvictRelUnpinnedBuffers(), EvictUnpinnedBuffer(), ExtendBufferedRelShared(), FindAndDropRelationBuffers(), FlushBuffer(), FlushDatabaseBuffers(), FlushRelationBuffers(), FlushRelationsAllBuffers(), GetBufferFromRing(), GetVictimBuffer(), InvalidateBuffer(), InvalidateVictimBuffer(), IsBufferCleanupOK(), LockBufferForCleanup(), MarkBufferDirtyHint(), pg_buffercache_numa_pages(), pg_buffercache_pages(), ReadRecentBuffer(), StartBufferIO(), StrategyGetBuffer(), SyncOneBuffer(), TerminateBufferIO(), UnlockBuffers(), WaitIO(), and WakePinCountWaiter().

◆ MarkBufferDirty()

void MarkBufferDirty ( Buffer  buffer)

Definition at line 2952 of file bufmgr.c.

2953{
2954 BufferDesc *bufHdr;
2955 uint32 buf_state;
2956 uint32 old_buf_state;
2957
2958 if (!BufferIsValid(buffer))
2959 elog(ERROR, "bad buffer ID: %d", buffer);
2960
2961 if (BufferIsLocal(buffer))
2962 {
2963 MarkLocalBufferDirty(buffer);
2964 return;
2965 }
2966
2967 bufHdr = GetBufferDescriptor(buffer - 1);
2968
2969 Assert(BufferIsPinned(buffer));
2971 LW_EXCLUSIVE));
2972
2973 old_buf_state = pg_atomic_read_u32(&bufHdr->state);
2974 for (;;)
2975 {
2976 if (old_buf_state & BM_LOCKED)
2977 old_buf_state = WaitBufHdrUnlocked(bufHdr);
2978
2979 buf_state = old_buf_state;
2980
2981 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
2982 buf_state |= BM_DIRTY | BM_JUST_DIRTIED;
2983
2984 if (pg_atomic_compare_exchange_u32(&bufHdr->state, &old_buf_state,
2985 buf_state))
2986 break;
2987 }
2988
2989 /*
2990 * If the buffer was not dirty already, do vacuum accounting.
2991 */
2992 if (!(old_buf_state & BM_DIRTY))
2993 {
2995 if (VacuumCostActive)
2997 }
2998}
static bool pg_atomic_compare_exchange_u32(volatile pg_atomic_uint32 *ptr, uint32 *expected, uint32 newval)
Definition: atomics.h:349
static uint32 WaitBufHdrUnlocked(BufferDesc *buf)
Definition: bufmgr.c:6287
int VacuumCostPageDirty
Definition: globals.c:154
void MarkLocalBufferDirty(Buffer buffer)
Definition: localbuf.c:489
int64 shared_blks_dirtied
Definition: instrument.h:28

References Assert(), BM_DIRTY, BM_JUST_DIRTIED, BM_LOCKED, BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferIsLocal, BufferIsPinned, BufferIsValid(), elog, ERROR, GetBufferDescriptor(), LW_EXCLUSIVE, LWLockHeldByMeInMode(), MarkLocalBufferDirty(), pg_atomic_compare_exchange_u32(), pg_atomic_read_u32(), pgBufferUsage, BufferUsage::shared_blks_dirtied, BufferDesc::state, VacuumCostActive, VacuumCostBalance, VacuumCostPageDirty, and WaitBufHdrUnlocked().

Referenced by _bt_clear_incomplete_split(), _bt_dedup_pass(), _bt_delitems_delete(), _bt_delitems_vacuum(), _bt_getroot(), _bt_insertonpg(), _bt_mark_page_halfdead(), _bt_newlevel(), _bt_restore_meta(), _bt_set_cleanup_info(), _bt_split(), _bt_unlink_halfdead_page(), _hash_addovflpage(), _hash_doinsert(), _hash_expandtable(), _hash_freeovflpage(), _hash_init(), _hash_splitbucket(), _hash_squeezebucket(), _hash_vacuum_one_page(), addLeafTuple(), brin_doinsert(), brin_doupdate(), brin_initialize_empty_new_buffer(), brin_xlog_createidx(), brin_xlog_desummarize_page(), brin_xlog_insert_update(), brin_xlog_revmap_extend(), brin_xlog_samepage_update(), brin_xlog_update(), brinbuild(), brinbuildempty(), brinRevmapDesummarizeRange(), btree_xlog_dedup(), btree_xlog_delete(), btree_xlog_insert(), btree_xlog_mark_page_halfdead(), btree_xlog_newroot(), btree_xlog_split(), btree_xlog_unlink_page(), btree_xlog_vacuum(), createPostingTree(), dataExecPlaceToPageInternal(), dataExecPlaceToPageLeaf(), do_setval(), doPickSplit(), entryExecPlaceToPage(), fill_seq_fork_with_data(), FreeSpaceMapPrepareTruncateRel(), generic_redo(), GenericXLogFinish(), ginbuild(), ginbuildempty(), ginbulkdelete(), ginDeletePage(), ginHeapTupleFastInsert(), ginPlaceToPage(), ginRedoClearIncompleteSplit(), ginRedoCreatePTree(), ginRedoDeleteListPages(), ginRedoDeletePage(), ginRedoInsert(), ginRedoInsertListPage(), ginRedoUpdateMetapage(), ginRedoVacuumDataLeafPage(), ginUpdateStats(), ginVacuumPostingTreeLeaf(), gistbuild(), gistbuildempty(), gistdeletepage(), gistplacetopage(), gistprunepage(), gistRedoClearFollowRight(), gistRedoDeleteRecord(), gistRedoPageDelete(), gistRedoPageSplitRecord(), gistRedoPageUpdateRecord(), gistvacuumpage(), hash_xlog_add_ovfl_page(), hash_xlog_delete(), hash_xlog_init_bitmap_page(), hash_xlog_init_meta_page(), hash_xlog_insert(), hash_xlog_move_page_contents(), hash_xlog_split_allocate_page(), hash_xlog_split_cleanup(), hash_xlog_split_complete(), hash_xlog_squeeze_page(), hash_xlog_update_meta_page(), hash_xlog_vacuum_one_page(), hashbucketcleanup(), hashbulkdelete(), heap_abort_speculative(), heap_delete(), heap_finish_speculative(), heap_force_common(), heap_inplace_update_and_unlock(), heap_insert(), heap_lock_tuple(), heap_lock_updated_tuple_rec(), heap_multi_insert(), heap_page_prune_and_freeze(), heap_update(), heap_xlog_confirm(), heap_xlog_delete(), heap_xlog_inplace(), heap_xlog_insert(), heap_xlog_lock(), heap_xlog_lock_updated(), heap_xlog_multi_insert(), heap_xlog_prune_freeze(), heap_xlog_update(), heap_xlog_visible(), lazy_scan_new_or_empty(), lazy_scan_prune(), lazy_vacuum_heap_page(), log_newpage_range(), moveLeafs(), nextval_internal(), RelationAddBlocks(), RelationCopyStorageUsingBuffer(), RelationGetBufferForTuple(), revmap_physical_extend(), saveNodeLink(), seq_redo(), shiftList(), spgAddNodeAction(), spgbuild(), SpGistUpdateMetaPage(), spgRedoAddLeaf(), spgRedoAddNode(), spgRedoMoveLeafs(), spgRedoPickSplit(), spgRedoSplitTuple(), spgRedoVacuumLeaf(), spgRedoVacuumRedirect(), spgRedoVacuumRoot(), spgSplitNodeAction(), vacuumLeafPage(), vacuumLeafRoot(), vacuumRedirectAndPlaceholder(), visibilitymap_clear(), visibilitymap_prepare_truncate(), visibilitymap_set(), writeListPage(), and XLogReadBufferForRedoExtended().

◆ MarkBufferDirtyHint()

void MarkBufferDirtyHint ( Buffer  buffer,
bool  buffer_std 
)

Definition at line 5435 of file bufmgr.c.

5436{
5437 BufferDesc *bufHdr;
5438 Page page = BufferGetPage(buffer);
5439
5440 if (!BufferIsValid(buffer))
5441 elog(ERROR, "bad buffer ID: %d", buffer);
5442
5443 if (BufferIsLocal(buffer))
5444 {
5445 MarkLocalBufferDirty(buffer);
5446 return;
5447 }
5448
5449 bufHdr = GetBufferDescriptor(buffer - 1);
5450
5451 Assert(GetPrivateRefCount(buffer) > 0);
5452 /* here, either share or exclusive lock is OK */
5454
5455 /*
5456 * This routine might get called many times on the same page, if we are
5457 * making the first scan after commit of an xact that added/deleted many
5458 * tuples. So, be as quick as we can if the buffer is already dirty. We
5459 * do this by not acquiring spinlock if it looks like the status bits are
5460 * already set. Since we make this test unlocked, there's a chance we
5461 * might fail to notice that the flags have just been cleared, and failed
5462 * to reset them, due to memory-ordering issues. But since this function
5463 * is only intended to be used in cases where failing to write out the
5464 * data would be harmless anyway, it doesn't really matter.
5465 */
5466 if ((pg_atomic_read_u32(&bufHdr->state) & (BM_DIRTY | BM_JUST_DIRTIED)) !=
5468 {
5470 bool dirtied = false;
5471 bool delayChkptFlags = false;
5472 uint32 buf_state;
5473
5474 /*
5475 * If we need to protect hint bit updates from torn writes, WAL-log a
5476 * full page image of the page. This full page image is only necessary
5477 * if the hint bit update is the first change to the page since the
5478 * last checkpoint.
5479 *
5480 * We don't check full_page_writes here because that logic is included
5481 * when we call XLogInsert() since the value changes dynamically.
5482 */
5483 if (XLogHintBitIsNeeded() &&
5485 {
5486 /*
5487 * If we must not write WAL, due to a relfilelocator-specific
5488 * condition or being in recovery, don't dirty the page. We can
5489 * set the hint, just not dirty the page as a result so the hint
5490 * is lost when we evict the page or shutdown.
5491 *
5492 * See src/backend/storage/page/README for longer discussion.
5493 */
5494 if (RecoveryInProgress() ||
5496 return;
5497
5498 /*
5499 * If the block is already dirty because we either made a change
5500 * or set a hint already, then we don't need to write a full page
5501 * image. Note that aggressive cleaning of blocks dirtied by hint
5502 * bit setting would increase the call rate. Bulk setting of hint
5503 * bits would reduce the call rate...
5504 *
5505 * We must issue the WAL record before we mark the buffer dirty.
5506 * Otherwise we might write the page before we write the WAL. That
5507 * causes a race condition, since a checkpoint might occur between
5508 * writing the WAL record and marking the buffer dirty. We solve
5509 * that with a kluge, but one that is already in use during
5510 * transaction commit to prevent race conditions. Basically, we
5511 * simply prevent the checkpoint WAL record from being written
5512 * until we have marked the buffer dirty. We don't start the
5513 * checkpoint flush until we have marked dirty, so our checkpoint
5514 * must flush the change to disk successfully or the checkpoint
5515 * never gets written, so crash recovery will fix.
5516 *
5517 * It's possible we may enter here without an xid, so it is
5518 * essential that CreateCheckPoint waits for virtual transactions
5519 * rather than full transactionids.
5520 */
5523 delayChkptFlags = true;
5524 lsn = XLogSaveBufferForHint(buffer, buffer_std);
5525 }
5526
5527 buf_state = LockBufHdr(bufHdr);
5528
5529 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
5530
5531 if (!(buf_state & BM_DIRTY))
5532 {
5533 dirtied = true; /* Means "will be dirtied by this action" */
5534
5535 /*
5536 * Set the page LSN if we wrote a backup block. We aren't supposed
5537 * to set this when only holding a share lock but as long as we
5538 * serialise it somehow we're OK. We choose to set LSN while
5539 * holding the buffer header lock, which causes any reader of an
5540 * LSN who holds only a share lock to also obtain a buffer header
5541 * lock before using PageGetLSN(), which is enforced in
5542 * BufferGetLSNAtomic().
5543 *
5544 * If checksums are enabled, you might think we should reset the
5545 * checksum here. That will happen when the page is written
5546 * sometime later in this checkpoint cycle.
5547 */
5548 if (!XLogRecPtrIsInvalid(lsn))
5549 PageSetLSN(page, lsn);
5550 }
5551
5552 buf_state |= BM_DIRTY | BM_JUST_DIRTIED;
5553 UnlockBufHdr(bufHdr, buf_state);
5554
5555 if (delayChkptFlags)
5556 MyProc->delayChkptFlags &= ~DELAY_CHKPT_START;
5557
5558 if (dirtied)
5559 {
5561 if (VacuumCostActive)
5563 }
5564 }
5565}
static void PageSetLSN(Page page, XLogRecPtr lsn)
Definition: bufpage.h:391
#define DELAY_CHKPT_START
Definition: proc.h:128
bool RelFileLocatorSkippingWAL(RelFileLocator rlocator)
Definition: storage.c:573
int delayChkptFlags
Definition: proc.h:249
bool RecoveryInProgress(void)
Definition: xlog.c:6522
#define XLogRecPtrIsInvalid(r)
Definition: xlogdefs.h:29
#define InvalidXLogRecPtr
Definition: xlogdefs.h:28
XLogRecPtr XLogSaveBufferForHint(Buffer buffer, bool buffer_std)
Definition: xloginsert.c:1065

References Assert(), BM_DIRTY, BM_JUST_DIRTIED, BM_PERMANENT, BUF_STATE_GET_REFCOUNT, PrivateRefCountEntry::buffer, BufferDescriptorGetContentLock(), BufferGetPage(), BufferIsLocal, BufferIsValid(), BufTagGetRelFileLocator(), DELAY_CHKPT_START, PGPROC::delayChkptFlags, elog, ERROR, GetBufferDescriptor(), GetPrivateRefCount(), InvalidXLogRecPtr, LockBufHdr(), LWLockHeldByMe(), MarkLocalBufferDirty(), MyProc, PageSetLSN(), pg_atomic_read_u32(), pgBufferUsage, RecoveryInProgress(), RelFileLocatorSkippingWAL(), BufferUsage::shared_blks_dirtied, BufferDesc::state, BufferDesc::tag, UnlockBufHdr(), VacuumCostActive, VacuumCostBalance, VacuumCostPageDirty, XLogHintBitIsNeeded, XLogRecPtrIsInvalid, and XLogSaveBufferForHint().

Referenced by _bt_check_unique(), _bt_killitems(), _hash_kill_items(), brin_start_evacuating_page(), btvacuumpage(), fsm_search(), fsm_search_avail(), fsm_set_and_search(), fsm_vacuum_page(), gistkillitems(), heap_page_prune_and_freeze(), read_seq_tuple(), SetHintBits(), and XLogRecordPageWithFreeSpace().

◆ NewPrivateRefCountEntry()

static PrivateRefCountEntry * NewPrivateRefCountEntry ( Buffer  buffer)
static

Definition at line 325 of file bufmgr.c.

326{
328
329 /* only allowed to be called when a reservation has been made */
331
332 /* use up the reserved entry */
335
336 /* and fill it */
337 res->buffer = buffer;
338 res->refcount = 0;
339
340 return res;
341}

References Assert(), PrivateRefCountEntry::buffer, PrivateRefCountEntry::refcount, and ReservedRefCountEntry.

Referenced by PinBuffer(), and PinBuffer_Locked().

◆ PinBuffer()

static bool PinBuffer ( BufferDesc buf,
BufferAccessStrategy  strategy 
)
static

Definition at line 3072 of file bufmgr.c.

3073{
3075 bool result;
3077
3080
3081 ref = GetPrivateRefCountEntry(b, true);
3082
3083 if (ref == NULL)
3084 {
3085 uint32 buf_state;
3086 uint32 old_buf_state;
3087
3089
3090 old_buf_state = pg_atomic_read_u32(&buf->state);
3091 for (;;)
3092 {
3093 if (old_buf_state & BM_LOCKED)
3094 old_buf_state = WaitBufHdrUnlocked(buf);
3095
3096 buf_state = old_buf_state;
3097
3098 /* increase refcount */
3099 buf_state += BUF_REFCOUNT_ONE;
3100
3101 if (strategy == NULL)
3102 {
3103 /* Default case: increase usagecount unless already max. */
3105 buf_state += BUF_USAGECOUNT_ONE;
3106 }
3107 else
3108 {
3109 /*
3110 * Ring buffers shouldn't evict others from pool. Thus we
3111 * don't make usagecount more than 1.
3112 */
3113 if (BUF_STATE_GET_USAGECOUNT(buf_state) == 0)
3114 buf_state += BUF_USAGECOUNT_ONE;
3115 }
3116
3117 if (pg_atomic_compare_exchange_u32(&buf->state, &old_buf_state,
3118 buf_state))
3119 {
3120 result = (buf_state & BM_VALID) != 0;
3121
3122 /*
3123 * Assume that we acquired a buffer pin for the purposes of
3124 * Valgrind buffer client checks (even in !result case) to
3125 * keep things simple. Buffers that are unsafe to access are
3126 * not generally guaranteed to be marked undefined or
3127 * non-accessible in any case.
3128 */
3130 break;
3131 }
3132 }
3133 }
3134 else
3135 {
3136 /*
3137 * If we previously pinned the buffer, it is likely to be valid, but
3138 * it may not be if StartReadBuffers() was called and
3139 * WaitReadBuffers() hasn't been called yet. We'll check by loading
3140 * the flags without locking. This is racy, but it's OK to return
3141 * false spuriously: when WaitReadBuffers() calls StartBufferIO(),
3142 * it'll see that it's now valid.
3143 *
3144 * Note: We deliberately avoid a Valgrind client request here.
3145 * Individual access methods can optionally superimpose buffer page
3146 * client requests on top of our client requests to enforce that
3147 * buffers are only accessed while locked (and pinned). It's possible
3148 * that the buffer page is legitimately non-accessible here. We
3149 * cannot meddle with that.
3150 */
3151 result = (pg_atomic_read_u32(&buf->state) & BM_VALID) != 0;
3152 }
3153
3154 ref->refcount++;
3155 Assert(ref->refcount > 0);
3157 return result;
3158}
#define BM_MAX_USAGE_COUNT
Definition: buf_internals.h:86
#define BUF_STATE_GET_USAGECOUNT(state)
Definition: buf_internals.h:60
static PrivateRefCountEntry * NewPrivateRefCountEntry(Buffer buffer)
Definition: bufmgr.c:325

References Assert(), b, BM_LOCKED, BM_MAX_USAGE_COUNT, BM_VALID, buf, BUF_REFCOUNT_ONE, BUF_STATE_GET_USAGECOUNT, BUF_USAGECOUNT_ONE, BufferDescriptorGetBuffer(), BufferIsLocal, BufHdrGetBlock, CurrentResourceOwner, GetPrivateRefCountEntry(), NewPrivateRefCountEntry(), pg_atomic_compare_exchange_u32(), pg_atomic_read_u32(), PrivateRefCountEntry::refcount, ReservedRefCountEntry, ResourceOwnerRememberBuffer(), VALGRIND_MAKE_MEM_DEFINED, and WaitBufHdrUnlocked().

Referenced by BufferAlloc(), ExtendBufferedRelShared(), and ReadRecentBuffer().

◆ PinBuffer_Locked()

static void PinBuffer_Locked ( BufferDesc buf)
static

Definition at line 3183 of file bufmgr.c.

3184{
3185 Buffer b;
3187 uint32 buf_state;
3188
3189 /*
3190 * As explained, We don't expect any preexisting pins. That allows us to
3191 * manipulate the PrivateRefCount after releasing the spinlock
3192 */
3194
3195 /*
3196 * Buffer can't have a preexisting pin, so mark its page as defined to
3197 * Valgrind (this is similar to the PinBuffer() case where the backend
3198 * doesn't already have a buffer pin)
3199 */
3201
3202 /*
3203 * Since we hold the buffer spinlock, we can update the buffer state and
3204 * release the lock in one operation.
3205 */
3206 buf_state = pg_atomic_read_u32(&buf->state);
3207 Assert(buf_state & BM_LOCKED);
3208 buf_state += BUF_REFCOUNT_ONE;
3209 UnlockBufHdr(buf, buf_state);
3210
3212
3214 ref->refcount++;
3215
3217}

References Assert(), b, BM_LOCKED, buf, BUF_REFCOUNT_ONE, BufferDescriptorGetBuffer(), BufHdrGetBlock, CurrentResourceOwner, GetPrivateRefCountEntry(), NewPrivateRefCountEntry(), pg_atomic_read_u32(), PrivateRefCountEntry::refcount, ResourceOwnerRememberBuffer(), UnlockBufHdr(), and VALGRIND_MAKE_MEM_DEFINED.

Referenced by EvictUnpinnedBufferInternal(), FlushDatabaseBuffers(), FlushRelationBuffers(), FlushRelationsAllBuffers(), GetVictimBuffer(), ReadRecentBuffer(), and SyncOneBuffer().

◆ PinBufferForBlock()

static pg_attribute_always_inline Buffer PinBufferForBlock ( Relation  rel,
SMgrRelation  smgr,
char  persistence,
ForkNumber  forkNum,
BlockNumber  blockNum,
BufferAccessStrategy  strategy,
bool *  foundPtr 
)
static

Definition at line 1110 of file bufmgr.c.

1117{
1118 BufferDesc *bufHdr;
1119 IOContext io_context;
1120 IOObject io_object;
1121
1122 Assert(blockNum != P_NEW);
1123
1124 /* Persistence should be set before */
1125 Assert((persistence == RELPERSISTENCE_TEMP ||
1126 persistence == RELPERSISTENCE_PERMANENT ||
1127 persistence == RELPERSISTENCE_UNLOGGED));
1128
1129 if (persistence == RELPERSISTENCE_TEMP)
1130 {
1131 io_context = IOCONTEXT_NORMAL;
1132 io_object = IOOBJECT_TEMP_RELATION;
1133 }
1134 else
1135 {
1136 io_context = IOContextForStrategy(strategy);
1137 io_object = IOOBJECT_RELATION;
1138 }
1139
1140 TRACE_POSTGRESQL_BUFFER_READ_START(forkNum, blockNum,
1144 smgr->smgr_rlocator.backend);
1145
1146 if (persistence == RELPERSISTENCE_TEMP)
1147 {
1148 bufHdr = LocalBufferAlloc(smgr, forkNum, blockNum, foundPtr);
1149 if (*foundPtr)
1151 }
1152 else
1153 {
1154 bufHdr = BufferAlloc(smgr, persistence, forkNum, blockNum,
1155 strategy, foundPtr, io_context);
1156 if (*foundPtr)
1158 }
1159 if (rel)
1160 {
1161 /*
1162 * While pgBufferUsage's "read" counter isn't bumped unless we reach
1163 * WaitReadBuffers() (so, not for hits, and not for buffers that are
1164 * zeroed instead), the per-relation stats always count them.
1165 */
1167 if (*foundPtr)
1169 }
1170 if (*foundPtr)
1171 {
1172 pgstat_count_io_op(io_object, io_context, IOOP_HIT, 1, 0);
1173 if (VacuumCostActive)
1175
1176 TRACE_POSTGRESQL_BUFFER_READ_DONE(forkNum, blockNum,
1180 smgr->smgr_rlocator.backend,
1181 true);
1182 }
1183
1184 return BufferDescriptorGetBuffer(bufHdr);
1185}
static BufferDesc * BufferAlloc(SMgrRelation smgr, char relpersistence, ForkNumber forkNum, BlockNumber blockNum, BufferAccessStrategy strategy, bool *foundPtr, IOContext io_context)
Definition: bufmgr.c:2005
#define P_NEW
Definition: bufmgr.h:191
BufferDesc * LocalBufferAlloc(SMgrRelation smgr, ForkNumber forkNum, BlockNumber blockNum, bool *foundPtr)
Definition: localbuf.c:118
#define pgstat_count_buffer_read(rel)
Definition: pgstat.h:704

References Assert(), RelFileLocatorBackend::backend, BufferAlloc(), BufferDescriptorGetBuffer(), RelFileLocator::dbOid, IOCONTEXT_NORMAL, IOContextForStrategy(), IOOBJECT_RELATION, IOOBJECT_TEMP_RELATION, IOOP_HIT, BufferUsage::local_blks_hit, LocalBufferAlloc(), RelFileLocatorBackend::locator, P_NEW, pgBufferUsage, pgstat_count_buffer_hit, pgstat_count_buffer_read, pgstat_count_io_op(), RelFileLocator::relNumber, BufferUsage::shared_blks_hit, SMgrRelationData::smgr_rlocator, RelFileLocator::spcOid, VacuumCostActive, VacuumCostBalance, and VacuumCostPageHit.

Referenced by ReadBuffer_common(), and StartReadBuffersImpl().

◆ PrefetchBuffer()

PrefetchBufferResult PrefetchBuffer ( Relation  reln,
ForkNumber  forkNum,
BlockNumber  blockNum 
)

Definition at line 651 of file bufmgr.c.

652{
653 Assert(RelationIsValid(reln));
654 Assert(BlockNumberIsValid(blockNum));
655
656 if (RelationUsesLocalBuffers(reln))
657 {
658 /* see comments in ReadBufferExtended */
659 if (RELATION_IS_OTHER_TEMP(reln))
661 (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
662 errmsg("cannot access temporary tables of other sessions")));
663
664 /* pass it off to localbuf.c */
665 return PrefetchLocalBuffer(RelationGetSmgr(reln), forkNum, blockNum);
666 }
667 else
668 {
669 /* pass it to the shared buffer version */
670 return PrefetchSharedBuffer(RelationGetSmgr(reln), forkNum, blockNum);
671 }
672}
PrefetchBufferResult PrefetchSharedBuffer(SMgrRelation smgr_reln, ForkNumber forkNum, BlockNumber blockNum)
Definition: bufmgr.c:561
PrefetchBufferResult PrefetchLocalBuffer(SMgrRelation smgr, ForkNumber forkNum, BlockNumber blockNum)
Definition: localbuf.c:71
#define RELATION_IS_OTHER_TEMP(relation)
Definition: rel.h:669
#define RelationIsValid(relation)
Definition: rel.h:489

References Assert(), BlockNumberIsValid(), ereport, errcode(), errmsg(), ERROR, PrefetchLocalBuffer(), PrefetchSharedBuffer(), RELATION_IS_OTHER_TEMP, RelationGetSmgr(), RelationIsValid, and RelationUsesLocalBuffers.

Referenced by count_nondeletable_pages(), invalidate_rel_block(), and pg_prewarm().

◆ PrefetchSharedBuffer()

PrefetchBufferResult PrefetchSharedBuffer ( SMgrRelation  smgr_reln,
ForkNumber  forkNum,
BlockNumber  blockNum 
)

Definition at line 561 of file bufmgr.c.

564{
565 PrefetchBufferResult result = {InvalidBuffer, false};
566 BufferTag newTag; /* identity of requested block */
567 uint32 newHash; /* hash value for newTag */
568 LWLock *newPartitionLock; /* buffer partition lock for it */
569 int buf_id;
570
571 Assert(BlockNumberIsValid(blockNum));
572
573 /* create a tag so we can lookup the buffer */
574 InitBufferTag(&newTag, &smgr_reln->smgr_rlocator.locator,
575 forkNum, blockNum);
576
577 /* determine its hash code and partition lock ID */
578 newHash = BufTableHashCode(&newTag);
579 newPartitionLock = BufMappingPartitionLock(newHash);
580
581 /* see if the block is in the buffer pool already */
582 LWLockAcquire(newPartitionLock, LW_SHARED);
583 buf_id = BufTableLookup(&newTag, newHash);
584 LWLockRelease(newPartitionLock);
585
586 /* If not in buffers, initiate prefetch */
587 if (buf_id < 0)
588 {
589#ifdef USE_PREFETCH
590 /*
591 * Try to initiate an asynchronous read. This returns false in
592 * recovery if the relation file doesn't exist.
593 */
594 if ((io_direct_flags & IO_DIRECT_DATA) == 0 &&
595 smgrprefetch(smgr_reln, forkNum, blockNum, 1))
596 {
597 result.initiated_io = true;
598 }
599#endif /* USE_PREFETCH */
600 }
601 else
602 {
603 /*
604 * Report the buffer it was in at that time. The caller may be able
605 * to avoid a buffer table lookup, but it's not pinned and it must be
606 * rechecked!
607 */
608 result.recent_buffer = buf_id + 1;
609 }
610
611 /*
612 * If the block *is* in buffers, we do nothing. This is not really ideal:
613 * the block might be just about to be evicted, which would be stupid
614 * since we know we are going to need it soon. But the only easy answer
615 * is to bump the usage_count, which does not seem like a great solution:
616 * when the caller does ultimately touch the block, usage_count would get
617 * bumped again, resulting in too much favoritism for blocks that are
618 * involved in a prefetch sequence. A real fix would involve some
619 * additional per-buffer state, and it's not clear that there's enough of
620 * a problem to justify that.
621 */
622
623 return result;
624}
int io_direct_flags
Definition: fd.c:168
#define IO_DIRECT_DATA
Definition: fd.h:54
bool smgrprefetch(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, int nblocks)
Definition: smgr.c:678
Buffer recent_buffer
Definition: bufmgr.h:61

References Assert(), BlockNumberIsValid(), BufMappingPartitionLock(), BufTableHashCode(), BufTableLookup(), InitBufferTag(), PrefetchBufferResult::initiated_io, InvalidBuffer, IO_DIRECT_DATA, io_direct_flags, RelFileLocatorBackend::locator, LW_SHARED, LWLockAcquire(), LWLockRelease(), PrefetchBufferResult::recent_buffer, SMgrRelationData::smgr_rlocator, and smgrprefetch().

Referenced by PrefetchBuffer(), and XLogPrefetcherNextBlock().

◆ ProcessReadBuffersResult()

static void ProcessReadBuffersResult ( ReadBuffersOperation operation)
static

Definition at line 1598 of file bufmgr.c.

1599{
1600 PgAioReturn *aio_ret = &operation->io_return;
1601 PgAioResultStatus rs = aio_ret->result.status;
1602 int newly_read_blocks = 0;
1603
1604 Assert(pgaio_wref_valid(&operation->io_wref));
1605 Assert(aio_ret->result.status != PGAIO_RS_UNKNOWN);
1606
1607 /*
1608 * SMGR reports the number of blocks successfully read as the result of
1609 * the IO operation. Thus we can simply add that to ->nblocks_done.
1610 */
1611
1612 if (likely(rs != PGAIO_RS_ERROR))
1613 newly_read_blocks = aio_ret->result.result;
1614
1615 if (rs == PGAIO_RS_ERROR || rs == PGAIO_RS_WARNING)
1616 pgaio_result_report(aio_ret->result, &aio_ret->target_data,
1617 rs == PGAIO_RS_ERROR ? ERROR : WARNING);
1618 else if (aio_ret->result.status == PGAIO_RS_PARTIAL)
1619 {
1620 /*
1621 * We'll retry, so we just emit a debug message to the server log (or
1622 * not even that in prod scenarios).
1623 */
1624 pgaio_result_report(aio_ret->result, &aio_ret->target_data, DEBUG1);
1625 elog(DEBUG3, "partial read, will retry");
1626 }
1627
1628 Assert(newly_read_blocks > 0);
1629 Assert(newly_read_blocks <= MAX_IO_COMBINE_LIMIT);
1630
1631 operation->nblocks_done += newly_read_blocks;
1632
1633 Assert(operation->nblocks_done <= operation->nblocks);
1634}
bool pgaio_wref_valid(PgAioWaitRef *iow)
Definition: aio.c:951
PgAioResultStatus
Definition: aio_types.h:79
@ PGAIO_RS_UNKNOWN
Definition: aio_types.h:80
@ PGAIO_RS_PARTIAL
Definition: aio_types.h:82
#define likely(x)
Definition: c.h:346
#define DEBUG3
Definition: elog.h:28
PgAioResult result
Definition: aio_types.h:132
PgAioTargetData target_data
Definition: aio_types.h:133

References Assert(), DEBUG1, DEBUG3, elog, ERROR, ReadBuffersOperation::io_return, ReadBuffersOperation::io_wref, likely, MAX_IO_COMBINE_LIMIT, ReadBuffersOperation::nblocks, ReadBuffersOperation::nblocks_done, pgaio_result_report(), PGAIO_RS_ERROR, PGAIO_RS_PARTIAL, PGAIO_RS_UNKNOWN, PGAIO_RS_WARNING, pgaio_wref_valid(), PgAioResult::result, PgAioReturn::result, PgAioResult::status, PgAioReturn::target_data, and WARNING.

Referenced by WaitReadBuffers().

◆ ReadBuffer()

Buffer ReadBuffer ( Relation  reln,
BlockNumber  blockNum 
)

Definition at line 758 of file bufmgr.c.

759{
760 return ReadBufferExtended(reln, MAIN_FORKNUM, blockNum, RBM_NORMAL, NULL);
761}
Buffer ReadBufferExtended(Relation reln, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy)
Definition: bufmgr.c:805
@ RBM_NORMAL
Definition: bufmgr.h:46

References MAIN_FORKNUM, RBM_NORMAL, and ReadBufferExtended().

Referenced by _bt_allocbuf(), _bt_getbuf(), _bt_search_insert(), _hash_getbuf(), _hash_getbuf_with_condlock_cleanup(), blbulkdelete(), blinsert(), BloomNewBuffer(), brin_getinsertbuffer(), brinGetStats(), brinGetTupleForHeapBlock(), brinRevmapDesummarizeRange(), brinRevmapInitialize(), bt_metap(), bt_multi_page_stats(), bt_page_items_internal(), bt_page_stats_internal(), ginFindLeafPage(), ginFindParents(), ginGetStats(), ginHeapTupleFastInsert(), ginInsertCleanup(), GinNewBuffer(), ginStepRight(), ginUpdateStats(), gistBufferingFindCorrectParent(), gistbufferinginserttuples(), gistdoinsert(), gistFindCorrectParent(), gistFindPath(), gistfixsplit(), gistGetMaxLevel(), gistkillitems(), gistNewBuffer(), gistProcessItup(), gistScanPage(), heap_abort_speculative(), heap_delete(), heap_fetch(), heap_finish_speculative(), heap_force_common(), heap_get_latest_tid(), heap_index_delete_tuples(), heap_lock_tuple(), heap_update(), initBloomState(), pg_visibility(), pgstatginindex_internal(), read_seq_tuple(), RelationGetBufferForTuple(), ReleaseAndReadBuffer(), revmap_get_buffer(), revmap_physical_extend(), scanGetCandidate(), scanPendingInsert(), shiftList(), spgdoinsert(), spgGetCache(), SpGistGetBuffer(), SpGistNewBuffer(), SpGistUpdateMetaPage(), and spgWalk().

◆ ReadBuffer_common()

static pg_attribute_always_inline Buffer ReadBuffer_common ( Relation  rel,
SMgrRelation  smgr,
char  smgr_persistence,
ForkNumber  forkNum,
BlockNumber  blockNum,
ReadBufferMode  mode,
BufferAccessStrategy  strategy 
)
static

Definition at line 1193 of file bufmgr.c.

1197{
1198 ReadBuffersOperation operation;
1199 Buffer buffer;
1200 int flags;
1201 char persistence;
1202
1203 /*
1204 * Backward compatibility path, most code should use ExtendBufferedRel()
1205 * instead, as acquiring the extension lock inside ExtendBufferedRel()
1206 * scales a lot better.
1207 */
1208 if (unlikely(blockNum == P_NEW))
1209 {
1211
1212 /*
1213 * Since no-one else can be looking at the page contents yet, there is
1214 * no difference between an exclusive lock and a cleanup-strength
1215 * lock.
1216 */
1218 flags |= EB_LOCK_FIRST;
1219
1220 return ExtendBufferedRel(BMR_REL(rel), forkNum, strategy, flags);
1221 }
1222
1223 if (rel)
1224 persistence = rel->rd_rel->relpersistence;
1225 else
1226 persistence = smgr_persistence;
1227
1230 {
1231 bool found;
1232
1233 buffer = PinBufferForBlock(rel, smgr, persistence,
1234 forkNum, blockNum, strategy, &found);
1235 ZeroAndLockBuffer(buffer, mode, found);
1236 return buffer;
1237 }
1238
1239 /*
1240 * Signal that we are going to immediately wait. If we're immediately
1241 * waiting, there is no benefit in actually executing the IO
1242 * asynchronously, it would just add dispatch overhead.
1243 */
1245 if (mode == RBM_ZERO_ON_ERROR)
1247 operation.smgr = smgr;
1248 operation.rel = rel;
1249 operation.persistence = persistence;
1250 operation.forknum = forkNum;
1251 operation.strategy = strategy;
1252 if (StartReadBuffer(&operation,
1253 &buffer,
1254 blockNum,
1255 flags))
1256 WaitReadBuffers(&operation);
1257
1258 return buffer;
1259}
Buffer ExtendBufferedRel(BufferManagerRelation bmr, ForkNumber forkNum, BufferAccessStrategy strategy, uint32 flags)
Definition: bufmgr.c:858
static void ZeroAndLockBuffer(Buffer buffer, ReadBufferMode mode, bool already_valid)
Definition: bufmgr.c:1031
static pg_attribute_always_inline Buffer PinBufferForBlock(Relation rel, SMgrRelation smgr, char persistence, ForkNumber forkNum, BlockNumber blockNum, BufferAccessStrategy strategy, bool *foundPtr)
Definition: bufmgr.c:1110
void WaitReadBuffers(ReadBuffersOperation *operation)
Definition: bufmgr.c:1637
bool StartReadBuffer(ReadBuffersOperation *operation, Buffer *buffer, BlockNumber blocknum, int flags)
Definition: bufmgr.c:1513
@ RBM_ZERO_ON_ERROR
Definition: bufmgr.h:51
#define BMR_REL(p_rel)
Definition: bufmgr.h:108

References BMR_REL, PrivateRefCountEntry::buffer, EB_LOCK_FIRST, EB_SKIP_EXTENSION_LOCK, ExtendBufferedRel(), ReadBuffersOperation::forknum, mode, P_NEW, ReadBuffersOperation::persistence, PinBufferForBlock(), RBM_ZERO_AND_CLEANUP_LOCK, RBM_ZERO_AND_LOCK, RBM_ZERO_ON_ERROR, RelationData::rd_rel, READ_BUFFERS_SYNCHRONOUSLY, READ_BUFFERS_ZERO_ON_ERROR, ReadBuffersOperation::rel, ReadBuffersOperation::smgr, StartReadBuffer(), ReadBuffersOperation::strategy, unlikely, WaitReadBuffers(), and ZeroAndLockBuffer().

Referenced by ExtendBufferedRelTo(), ReadBufferExtended(), and ReadBufferWithoutRelcache().

◆ ReadBufferExtended()

Buffer ReadBufferExtended ( Relation  reln,
ForkNumber  forkNum,
BlockNumber  blockNum,
ReadBufferMode  mode,
BufferAccessStrategy  strategy 
)
inline

Definition at line 805 of file bufmgr.c.

807{
808 Buffer buf;
809
810 /*
811 * Reject attempts to read non-local temporary relations; we would be
812 * likely to get wrong data since we have no visibility into the owning
813 * session's local buffers.
814 */
815 if (RELATION_IS_OTHER_TEMP(reln))
817 (errcode(ERRCODE_FEATURE_NOT_SUPPORTED),
818 errmsg("cannot access temporary tables of other sessions")));
819
820 /*
821 * Read the buffer, and update pgstat counters to reflect a cache hit or
822 * miss.
823 */
824 buf = ReadBuffer_common(reln, RelationGetSmgr(reln), 0,
825 forkNum, blockNum, mode, strategy);
826
827 return buf;
828}

References buf, ereport, errcode(), errmsg(), ERROR, mode, ReadBuffer_common(), RELATION_IS_OTHER_TEMP, and RelationGetSmgr().

Referenced by _hash_getbuf_with_strategy(), _hash_getinitbuf(), _hash_getnewbuf(), blbulkdelete(), blgetbitmap(), BloomInitMetapage(), blvacuumcleanup(), brin_vacuum_scan(), bt_recheck_sibling_links(), btvacuumpage(), count_nondeletable_pages(), create_toy_buffer(), fsm_readbuf(), get_raw_page_internal(), gin_check_parent_keys_consistency(), gin_check_posting_tree_parent_keys_consistency(), gin_refind_parent(), ginbulkdelete(), ginDeletePage(), ginScanToDelete(), ginvacuumcleanup(), ginVacuumPostingTree(), ginVacuumPostingTreeLeaves(), gistvacuum_delete_empty_pages(), gistvacuumpage(), hashbulkdelete(), heapam_scan_sample_next_block(), log_newpage_range(), modify_rel_block(), palloc_btree_page(), pgstat_btree_page(), pgstat_gist_page(), pgstat_heap(), pgstathashindex(), pgstatindex_impl(), ReadBuffer(), ReadBufferBI(), spgprocesspending(), statapprox_heap(), and vm_readbuf().

◆ ReadBuffersCanStartIO()

static bool ReadBuffersCanStartIO ( Buffer  buffer,
bool  nowait 
)
inlinestatic

Definition at line 1569 of file bufmgr.c.

1570{
1571 /*
1572 * If this backend currently has staged IO, we need to submit the pending
1573 * IO before waiting for the right to issue IO, to avoid the potential for
1574 * deadlocks (and, more commonly, unnecessary delays for other backends).
1575 */
1576 if (!nowait && pgaio_have_staged())
1577 {
1578 if (ReadBuffersCanStartIOOnce(buffer, true))
1579 return true;
1580
1581 /*
1582 * Unfortunately StartBufferIO() returning false doesn't allow to
1583 * distinguish between the buffer already being valid and IO already
1584 * being in progress. Since IO already being in progress is quite
1585 * rare, this approach seems fine.
1586 */
1588 }
1589
1590 return ReadBuffersCanStartIOOnce(buffer, nowait);
1591}
bool pgaio_have_staged(void)
Definition: aio.c:1087
static bool ReadBuffersCanStartIOOnce(Buffer buffer, bool nowait)
Definition: bufmgr.c:1556

References PrivateRefCountEntry::buffer, pgaio_have_staged(), pgaio_submit_staged(), and ReadBuffersCanStartIOOnce().

Referenced by AsyncReadBuffers().

◆ ReadBuffersCanStartIOOnce()

static bool ReadBuffersCanStartIOOnce ( Buffer  buffer,
bool  nowait 
)
inlinestatic

Definition at line 1556 of file bufmgr.c.

1557{
1558 if (BufferIsLocal(buffer))
1559 return StartLocalBufferIO(GetLocalBufferDescriptor(-buffer - 1),
1560 true, nowait);
1561 else
1562 return StartBufferIO(GetBufferDescriptor(buffer - 1), true, nowait);
1563}
bool StartLocalBufferIO(BufferDesc *bufHdr, bool forInput, bool nowait)
Definition: localbuf.c:521

References PrivateRefCountEntry::buffer, BufferIsLocal, GetBufferDescriptor(), GetLocalBufferDescriptor(), StartBufferIO(), and StartLocalBufferIO().

Referenced by ReadBuffersCanStartIO().

◆ ReadBufferWithoutRelcache()

Buffer ReadBufferWithoutRelcache ( RelFileLocator  rlocator,
ForkNumber  forkNum,
BlockNumber  blockNum,
ReadBufferMode  mode,
BufferAccessStrategy  strategy,
bool  permanent 
)

Definition at line 842 of file bufmgr.c.

845{
846 SMgrRelation smgr = smgropen(rlocator, INVALID_PROC_NUMBER);
847
848 return ReadBuffer_common(NULL, smgr,
849 permanent ? RELPERSISTENCE_PERMANENT : RELPERSISTENCE_UNLOGGED,
850 forkNum, blockNum,
851 mode, strategy);
852}

References INVALID_PROC_NUMBER, mode, ReadBuffer_common(), and smgropen().

Referenced by RelationCopyStorageUsingBuffer(), ScanSourceDatabasePgClass(), and XLogReadBufferExtended().

◆ ReadRecentBuffer()

bool ReadRecentBuffer ( RelFileLocator  rlocator,
ForkNumber  forkNum,
BlockNumber  blockNum,
Buffer  recent_buffer 
)

Definition at line 682 of file bufmgr.c.

684{
685 BufferDesc *bufHdr;
686 BufferTag tag;
687 uint32 buf_state;
688 bool have_private_ref;
689
690 Assert(BufferIsValid(recent_buffer));
691
694 InitBufferTag(&tag, &rlocator, forkNum, blockNum);
695
696 if (BufferIsLocal(recent_buffer))
697 {
698 int b = -recent_buffer - 1;
699
700 bufHdr = GetLocalBufferDescriptor(b);
701 buf_state = pg_atomic_read_u32(&bufHdr->state);
702
703 /* Is it still valid and holding the right tag? */
704 if ((buf_state & BM_VALID) && BufferTagsEqual(&tag, &bufHdr->tag))
705 {
706 PinLocalBuffer(bufHdr, true);
707
709
710 return true;
711 }
712 }
713 else
714 {
715 bufHdr = GetBufferDescriptor(recent_buffer - 1);
716 have_private_ref = GetPrivateRefCount(recent_buffer) > 0;
717
718 /*
719 * Do we already have this buffer pinned with a private reference? If
720 * so, it must be valid and it is safe to check the tag without
721 * locking. If not, we have to lock the header first and then check.
722 */
723 if (have_private_ref)
724 buf_state = pg_atomic_read_u32(&bufHdr->state);
725 else
726 buf_state = LockBufHdr(bufHdr);
727
728 if ((buf_state & BM_VALID) && BufferTagsEqual(&tag, &bufHdr->tag))
729 {
730 /*
731 * It's now safe to pin the buffer. We can't pin first and ask
732 * questions later, because it might confuse code paths like
733 * InvalidateBuffer() if we pinned a random non-matching buffer.
734 */
735 if (have_private_ref)
736 PinBuffer(bufHdr, NULL); /* bump pin count */
737 else
738 PinBuffer_Locked(bufHdr); /* pin for first time */
739
741
742 return true;
743 }
744
745 /* If we locked the header above, now unlock. */
746 if (!have_private_ref)
747 UnlockBufHdr(bufHdr, buf_state);
748 }
749
750 return false;
751}

References Assert(), b, BM_VALID, BufferIsLocal, BufferIsValid(), BufferTagsEqual(), CurrentResourceOwner, GetBufferDescriptor(), GetLocalBufferDescriptor(), GetPrivateRefCount(), InitBufferTag(), BufferUsage::local_blks_hit, LockBufHdr(), pg_atomic_read_u32(), pgBufferUsage, PinBuffer(), PinBuffer_Locked(), PinLocalBuffer(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), BufferUsage::shared_blks_hit, BufferDesc::state, BufferDesc::tag, and UnlockBufHdr().

Referenced by invalidate_rel_block(), and XLogReadBufferExtended().

◆ RelationCopyStorageUsingBuffer()

static void RelationCopyStorageUsingBuffer ( RelFileLocator  srclocator,
RelFileLocator  dstlocator,
ForkNumber  forkNum,
bool  permanent 
)
static

Definition at line 5131 of file bufmgr.c.

5134{
5135 Buffer srcBuf;
5136 Buffer dstBuf;
5137 Page srcPage;
5138 Page dstPage;
5139 bool use_wal;
5140 BlockNumber nblocks;
5141 BlockNumber blkno;
5143 BufferAccessStrategy bstrategy_src;
5144 BufferAccessStrategy bstrategy_dst;
5146 ReadStream *src_stream;
5147 SMgrRelation src_smgr;
5148
5149 /*
5150 * In general, we want to write WAL whenever wal_level > 'minimal', but we
5151 * can skip it when copying any fork of an unlogged relation other than
5152 * the init fork.
5153 */
5154 use_wal = XLogIsNeeded() && (permanent || forkNum == INIT_FORKNUM);
5155
5156 /* Get number of blocks in the source relation. */
5157 nblocks = smgrnblocks(smgropen(srclocator, INVALID_PROC_NUMBER),
5158 forkNum);
5159
5160 /* Nothing to copy; just return. */
5161 if (nblocks == 0)
5162 return;
5163
5164 /*
5165 * Bulk extend the destination relation of the same size as the source
5166 * relation before starting to copy block by block.
5167 */
5168 memset(buf.data, 0, BLCKSZ);
5169 smgrextend(smgropen(dstlocator, INVALID_PROC_NUMBER), forkNum, nblocks - 1,
5170 buf.data, true);
5171
5172 /* This is a bulk operation, so use buffer access strategies. */
5173 bstrategy_src = GetAccessStrategy(BAS_BULKREAD);
5174 bstrategy_dst = GetAccessStrategy(BAS_BULKWRITE);
5175
5176 /* Initialize streaming read */
5177 p.current_blocknum = 0;
5178 p.last_exclusive = nblocks;
5179 src_smgr = smgropen(srclocator, INVALID_PROC_NUMBER);
5180
5181 /*
5182 * It is safe to use batchmode as block_range_read_stream_cb takes no
5183 * locks.
5184 */
5187 bstrategy_src,
5188 src_smgr,
5189 permanent ? RELPERSISTENCE_PERMANENT : RELPERSISTENCE_UNLOGGED,
5190 forkNum,
5192 &p,
5193 0);
5194
5195 /* Iterate over each block of the source relation file. */
5196 for (blkno = 0; blkno < nblocks; blkno++)
5197 {
5199
5200 /* Read block from source relation. */
5201 srcBuf = read_stream_next_buffer(src_stream, NULL);
5203 srcPage = BufferGetPage(srcBuf);
5204
5205 dstBuf = ReadBufferWithoutRelcache(dstlocator, forkNum,
5206 BufferGetBlockNumber(srcBuf),
5207 RBM_ZERO_AND_LOCK, bstrategy_dst,
5208 permanent);
5209 dstPage = BufferGetPage(dstBuf);
5210
5212
5213 /* Copy page data from the source to the destination. */
5214 memcpy(dstPage, srcPage, BLCKSZ);
5215 MarkBufferDirty(dstBuf);
5216
5217 /* WAL-log the copied page. */
5218 if (use_wal)
5219 log_newpage_buffer(dstBuf, true);
5220
5222
5223 UnlockReleaseBuffer(dstBuf);
5224 UnlockReleaseBuffer(srcBuf);
5225 }
5226 Assert(read_stream_next_buffer(src_stream, NULL) == InvalidBuffer);
5227 read_stream_end(src_stream);
5228
5229 FreeAccessStrategy(bstrategy_src);
5230 FreeAccessStrategy(bstrategy_dst);
5231}
void UnlockReleaseBuffer(Buffer buffer)
Definition: bufmgr.c:5388
void MarkBufferDirty(Buffer buffer)
Definition: bufmgr.c:2952
Buffer ReadBufferWithoutRelcache(RelFileLocator rlocator, ForkNumber forkNum, BlockNumber blockNum, ReadBufferMode mode, BufferAccessStrategy strategy, bool permanent)
Definition: bufmgr.c:842
@ BAS_BULKREAD
Definition: bufmgr.h:37
@ BAS_BULKWRITE
Definition: bufmgr.h:39
BufferAccessStrategy GetAccessStrategy(BufferAccessStrategyType btype)
Definition: freelist.c:541
void FreeAccessStrategy(BufferAccessStrategy strategy)
Definition: freelist.c:723
#define START_CRIT_SECTION()
Definition: miscadmin.h:150
#define CHECK_FOR_INTERRUPTS()
Definition: miscadmin.h:123
#define END_CRIT_SECTION()
Definition: miscadmin.h:152
ReadStream * read_stream_begin_smgr_relation(int flags, BufferAccessStrategy strategy, SMgrRelation smgr, char smgr_persistence, ForkNumber forknum, ReadStreamBlockNumberCB callback, void *callback_private_data, size_t per_buffer_data_size)
Definition: read_stream.c:740
Buffer read_stream_next_buffer(ReadStream *stream, void **per_buffer_data)
Definition: read_stream.c:770
void read_stream_end(ReadStream *stream)
Definition: read_stream.c:1055
BlockNumber block_range_read_stream_cb(ReadStream *stream, void *callback_private_data, void *per_buffer_data)
Definition: read_stream.c:162
#define READ_STREAM_USE_BATCHING
Definition: read_stream.h:64
#define READ_STREAM_FULL
Definition: read_stream.h:43
void smgrextend(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum, const void *buffer, bool skipFsync)
Definition: smgr.c:620
#define XLogIsNeeded()
Definition: xlog.h:109
XLogRecPtr log_newpage_buffer(Buffer buffer, bool page_std)
Definition: xloginsert.c:1237

References Assert(), BAS_BULKREAD, BAS_BULKWRITE, block_range_read_stream_cb(), buf, BUFFER_LOCK_SHARE, BufferGetBlockNumber(), BufferGetPage(), CHECK_FOR_INTERRUPTS, BlockRangeReadStreamPrivate::current_blocknum, END_CRIT_SECTION, FreeAccessStrategy(), GetAccessStrategy(), INIT_FORKNUM, INVALID_PROC_NUMBER, InvalidBuffer, BlockRangeReadStreamPrivate::last_exclusive, LockBuffer(), log_newpage_buffer(), MarkBufferDirty(), RBM_ZERO_AND_LOCK, read_stream_begin_smgr_relation(), read_stream_end(), READ_STREAM_FULL, read_stream_next_buffer(), READ_STREAM_USE_BATCHING, ReadBufferWithoutRelcache(), smgrextend(), smgrnblocks(), smgropen(), START_CRIT_SECTION, UnlockReleaseBuffer(), and XLogIsNeeded.

Referenced by CreateAndCopyRelationData().

◆ RelationGetNumberOfBlocksInFork()

BlockNumber RelationGetNumberOfBlocksInFork ( Relation  relation,
ForkNumber  forkNum 
)

Definition at line 4429 of file bufmgr.c.

4430{
4431 if (RELKIND_HAS_TABLE_AM(relation->rd_rel->relkind))
4432 {
4433 /*
4434 * Not every table AM uses BLCKSZ wide fixed size blocks. Therefore
4435 * tableam returns the size in bytes - but for the purpose of this
4436 * routine, we want the number of blocks. Therefore divide, rounding
4437 * up.
4438 */
4439 uint64 szbytes;
4440
4441 szbytes = table_relation_size(relation, forkNum);
4442
4443 return (szbytes + (BLCKSZ - 1)) / BLCKSZ;
4444 }
4445 else if (RELKIND_HAS_STORAGE(relation->rd_rel->relkind))
4446 {
4447 return smgrnblocks(RelationGetSmgr(relation), forkNum);
4448 }
4449 else
4450 Assert(false);
4451
4452 return 0; /* keep compiler quiet */
4453}
static uint64 table_relation_size(Relation rel, ForkNumber forkNumber)
Definition: tableam.h:1828

References Assert(), RelationData::rd_rel, RelationGetSmgr(), smgrnblocks(), and table_relation_size().

Referenced by _hash_getnewbuf(), _hash_init(), autoprewarm_database_main(), get_raw_page_internal(), and pg_prewarm().

◆ ReleaseAndReadBuffer()

Buffer ReleaseAndReadBuffer ( Buffer  buffer,
Relation  relation,
BlockNumber  blockNum 
)

Definition at line 3014 of file bufmgr.c.

3017{
3018 ForkNumber forkNum = MAIN_FORKNUM;
3019 BufferDesc *bufHdr;
3020
3021 if (BufferIsValid(buffer))
3022 {
3023 Assert(BufferIsPinned(buffer));
3024 if (BufferIsLocal(buffer))
3025 {
3026 bufHdr = GetLocalBufferDescriptor(-buffer - 1);
3027 if (bufHdr->tag.blockNum == blockNum &&
3028 BufTagMatchesRelFileLocator(&bufHdr->tag, &relation->rd_locator) &&
3029 BufTagGetForkNum(&bufHdr->tag) == forkNum)
3030 return buffer;
3031 UnpinLocalBuffer(buffer);
3032 }
3033 else
3034 {
3035 bufHdr = GetBufferDescriptor(buffer - 1);
3036 /* we have pin, so it's ok to examine tag without spinlock */
3037 if (bufHdr->tag.blockNum == blockNum &&
3038 BufTagMatchesRelFileLocator(&bufHdr->tag, &relation->rd_locator) &&
3039 BufTagGetForkNum(&bufHdr->tag) == forkNum)
3040 return buffer;
3041 UnpinBuffer(bufHdr);
3042 }
3043 }
3044
3045 return ReadBuffer(relation, blockNum);
3046}
Buffer ReadBuffer(Relation reln, BlockNumber blockNum)
Definition: bufmgr.c:758

References Assert(), buftag::blockNum, PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsPinned, BufferIsValid(), BufTagGetForkNum(), BufTagMatchesRelFileLocator(), GetBufferDescriptor(), GetLocalBufferDescriptor(), MAIN_FORKNUM, RelationData::rd_locator, ReadBuffer(), BufferDesc::tag, UnpinBuffer(), and UnpinLocalBuffer().

Referenced by _bt_relandgetbuf(), ginFindLeafPage(), and heapam_index_fetch_tuple().

◆ ReleaseBuffer()

void ReleaseBuffer ( Buffer  buffer)

Definition at line 5371 of file bufmgr.c.

5372{
5373 if (!BufferIsValid(buffer))
5374 elog(ERROR, "bad buffer ID: %d", buffer);
5375
5376 if (BufferIsLocal(buffer))
5377 UnpinLocalBuffer(buffer);
5378 else
5379 UnpinBuffer(GetBufferDescriptor(buffer - 1));
5380}

References PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsValid(), elog, ERROR, GetBufferDescriptor(), UnpinBuffer(), and UnpinLocalBuffer().

Referenced by _bt_allocbuf(), _bt_drop_lock_and_maybe_pin(), _bt_pagedel(), _bt_relbuf(), _bt_search_insert(), _bt_unlink_halfdead_page(), _hash_dropbuf(), _hash_getbuf_with_condlock_cleanup(), autoprewarm_database_main(), BitmapHeapScanNextBlock(), blinsert(), BloomNewBuffer(), brin_getinsertbuffer(), brin_vacuum_scan(), bringetbitmap(), brinGetTupleForHeapBlock(), brininsert(), brinRevmapTerminate(), brinsummarize(), buffer_create_toy(), collect_corrupt_items(), collect_visibility_data(), entryLoadMoreItems(), ExecEndIndexOnlyScan(), ExtendBufferedRelTo(), FreeBulkInsertState(), freeGinBtreeStack(), fsm_search(), fsm_vacuum_page(), get_actual_variable_endpoint(), get_raw_page_internal(), GetRecordedFreeSpace(), gin_check_parent_keys_consistency(), gin_check_posting_tree_parent_keys_consistency(), ginDeletePage(), ginFindParents(), ginFinishSplit(), ginFreeScanKeys(), ginInsertCleanup(), GinNewBuffer(), ginScanToDelete(), gistdoinsert(), gistFindCorrectParent(), gistNewBuffer(), gistvacuum_delete_empty_pages(), grow_rel(), heap_abort_speculative(), heap_delete(), heap_endscan(), heap_fetch(), heap_fetch_next_buffer(), heap_force_common(), heap_insert(), heap_lock_tuple(), heap_lock_updated_tuple_rec(), heap_multi_insert(), heap_rescan(), heap_update(), heap_vac_scan_next_block(), heap_xlog_delete(), heap_xlog_insert(), heap_xlog_lock(), heap_xlog_lock_updated(), heap_xlog_multi_insert(), heap_xlog_update(), heap_xlog_visible(), heapam_index_fetch_reset(), heapam_scan_sample_next_block(), heapam_tuple_lock(), heapgettup(), heapgettup_pagemode(), invalidate_rel_block(), lazy_scan_heap(), lazy_vacuum_heap_rel(), modify_rel_block(), pg_prewarm(), pg_visibility(), pg_visibility_map(), pg_visibility_map_summary(), pgstatindex_impl(), read_rel_block_ll(), read_stream_reset(), ReadBufferBI(), RelationAddBlocks(), RelationGetBufferForTuple(), ReleaseBulkInsertStatePin(), revmap_get_buffer(), spgdoinsert(), SpGistGetBuffer(), SpGistNewBuffer(), SpGistUpdateMetaPage(), statapprox_heap(), summarize_range(), terminate_brin_buildstate(), tts_buffer_heap_clear(), tts_buffer_heap_materialize(), tts_buffer_heap_store_tuple(), UnlockReleaseBuffer(), verify_heapam(), visibilitymap_count(), visibilitymap_get_status(), visibilitymap_pin(), and XLogReadBufferExtended().

◆ ReservePrivateRefCountEntry()

static void ReservePrivateRefCountEntry ( void  )
static

Definition at line 259 of file bufmgr.c.

260{
261 /* Already reserved (or freed), nothing to do */
262 if (ReservedRefCountEntry != NULL)
263 return;
264
265 /*
266 * First search for a free entry the array, that'll be sufficient in the
267 * majority of cases.
268 */
269 {
270 int i;
271
272 for (i = 0; i < REFCOUNT_ARRAY_ENTRIES; i++)
273 {
275
276 res = &PrivateRefCountArray[i];
277
278 if (res->buffer == InvalidBuffer)
279 {
281 return;
282 }
283 }
284 }
285
286 /*
287 * No luck. All array entries are full. Move one array entry into the hash
288 * table.
289 */
290 {
291 /*
292 * Move entry from the current clock position in the array into the
293 * hashtable. Use that slot.
294 */
295 PrivateRefCountEntry *hashent;
296 bool found;
297
298 /* select victim slot */
301
302 /* Better be used, otherwise we shouldn't get here. */
304
305 /* enter victim array entry into hashtable */
309 &found);
310 Assert(!found);
312
313 /* clear the now free array slot */
316
318 }
319}
static uint32 PrivateRefCountClock
Definition: bufmgr.c:218
@ HASH_ENTER
Definition: hsearch.h:114

References Assert(), PrivateRefCountEntry::buffer, HASH_ENTER, hash_search(), i, InvalidBuffer, PrivateRefCountArray, PrivateRefCountClock, PrivateRefCountHash, PrivateRefCountOverflowed, PrivateRefCountEntry::refcount, REFCOUNT_ARRAY_ENTRIES, and ReservedRefCountEntry.

Referenced by BufferAlloc(), EvictAllUnpinnedBuffers(), EvictRelUnpinnedBuffers(), EvictUnpinnedBuffer(), ExtendBufferedRelShared(), FlushDatabaseBuffers(), FlushRelationBuffers(), FlushRelationsAllBuffers(), GetPrivateRefCountEntry(), GetVictimBuffer(), ReadRecentBuffer(), and SyncOneBuffer().

◆ ResOwnerPrintBufferIO()

static char * ResOwnerPrintBufferIO ( Datum  res)
static

Definition at line 6552 of file bufmgr.c.

6553{
6554 Buffer buffer = DatumGetInt32(res);
6555
6556 return psprintf("lost track of buffer IO on buffer %d", buffer);
6557}
static int32 DatumGetInt32(Datum X)
Definition: postgres.h:207

References PrivateRefCountEntry::buffer, DatumGetInt32(), and psprintf().

◆ ResOwnerPrintBufferPin()

static char * ResOwnerPrintBufferPin ( Datum  res)
static

Definition at line 6575 of file bufmgr.c.

6576{
6578}

References DatumGetInt32(), and DebugPrintBufferRefcount().

◆ ResOwnerReleaseBufferIO()

static void ResOwnerReleaseBufferIO ( Datum  res)
static

Definition at line 6544 of file bufmgr.c.

6545{
6546 Buffer buffer = DatumGetInt32(res);
6547
6548 AbortBufferIO(buffer);
6549}
static void AbortBufferIO(Buffer buffer)
Definition: bufmgr.c:6159

References AbortBufferIO(), PrivateRefCountEntry::buffer, and DatumGetInt32().

◆ ResOwnerReleaseBufferPin()

static void ResOwnerReleaseBufferPin ( Datum  res)
static

Definition at line 6560 of file bufmgr.c.

6561{
6562 Buffer buffer = DatumGetInt32(res);
6563
6564 /* Like ReleaseBuffer, but don't call ResourceOwnerForgetBuffer */
6565 if (!BufferIsValid(buffer))
6566 elog(ERROR, "bad buffer ID: %d", buffer);
6567
6568 if (BufferIsLocal(buffer))
6570 else
6572}
static void UnpinBufferNoOwner(BufferDesc *buf)
Definition: bufmgr.c:3273
void UnpinLocalBufferNoOwner(Buffer buffer)
Definition: localbuf.c:839

References PrivateRefCountEntry::buffer, BufferIsLocal, BufferIsValid(), DatumGetInt32(), elog, ERROR, GetBufferDescriptor(), UnpinBufferNoOwner(), and UnpinLocalBufferNoOwner().

◆ rlocator_comparator()

static int rlocator_comparator ( const void *  p1,
const void *  p2 
)
static

Definition at line 6230 of file bufmgr.c.

6231{
6232 RelFileLocator n1 = *(const RelFileLocator *) p1;
6233 RelFileLocator n2 = *(const RelFileLocator *) p2;
6234
6235 if (n1.relNumber < n2.relNumber)
6236 return -1;
6237 else if (n1.relNumber > n2.relNumber)
6238 return 1;
6239
6240 if (n1.dbOid < n2.dbOid)
6241 return -1;
6242 else if (n1.dbOid > n2.dbOid)
6243 return 1;
6244
6245 if (n1.spcOid < n2.spcOid)
6246 return -1;
6247 else if (n1.spcOid > n2.spcOid)
6248 return 1;
6249 else
6250 return 0;
6251}

References RelFileLocator::dbOid, RelFileLocator::relNumber, and RelFileLocator::spcOid.

Referenced by buffertag_comparator(), DropRelationsAllBuffers(), and FlushRelationsAllBuffers().

◆ ScheduleBufferTagForWriteback()

void ScheduleBufferTagForWriteback ( WritebackContext wb_context,
IOContext  io_context,
BufferTag tag 
)

Definition at line 6411 of file bufmgr.c.

6413{
6414 PendingWriteback *pending;
6415
6416 /*
6417 * As pg_flush_data() doesn't do anything with fsync disabled, there's no
6418 * point in tracking in that case.
6419 */
6421 !enableFsync)
6422 return;
6423
6424 /*
6425 * Add buffer to the pending writeback array, unless writeback control is
6426 * disabled.
6427 */
6428 if (*wb_context->max_pending > 0)
6429 {
6431
6432 pending = &wb_context->pending_writebacks[wb_context->nr_pending++];
6433
6434 pending->tag = *tag;
6435 }
6436
6437 /*
6438 * Perform pending flushes if the writeback limit is exceeded. This
6439 * includes the case where previously an item has been added, but control
6440 * is now disabled.
6441 */
6442 if (wb_context->nr_pending >= *wb_context->max_pending)
6443 IssuePendingWritebacks(wb_context, io_context);
6444}
bool enableFsync
Definition: globals.c:130
#define WRITEBACK_MAX_PENDING_FLUSHES

References Assert(), enableFsync, IO_DIRECT_DATA, io_direct_flags, IssuePendingWritebacks(), WritebackContext::max_pending, WritebackContext::nr_pending, WritebackContext::pending_writebacks, PendingWriteback::tag, and WRITEBACK_MAX_PENDING_FLUSHES.

Referenced by GetVictimBuffer(), and SyncOneBuffer().

◆ shared_buffer_readv_complete()

static PgAioResult shared_buffer_readv_complete ( PgAioHandle ioh,
PgAioResult  prior_result,
uint8  cb_data 
)
static

Definition at line 7377 of file bufmgr.c.

7379{
7380 return buffer_readv_complete(ioh, prior_result, cb_data, false);
7381}

References buffer_readv_complete().

◆ shared_buffer_readv_complete_local()

static PgAioResult shared_buffer_readv_complete_local ( PgAioHandle ioh,
PgAioResult  prior_result,
uint8  cb_data 
)
static

Definition at line 7391 of file bufmgr.c.

7393{
7394 bool zeroed_any,
7395 ignored_any;
7396 uint8 zeroed_or_error_count,
7397 checkfail_count,
7398 first_off;
7399
7400 if (prior_result.status == PGAIO_RS_OK)
7401 return prior_result;
7402
7403 buffer_readv_decode_error(prior_result,
7404 &zeroed_any,
7405 &ignored_any,
7406 &zeroed_or_error_count,
7407 &checkfail_count,
7408 &first_off);
7409
7410 if (checkfail_count)
7411 {
7413
7415 checkfail_count);
7416 }
7417
7418 return prior_result;
7419}
@ PGAIO_RS_OK
Definition: aio_types.h:81

References buffer_readv_decode_error(), RelFileLocator::dbOid, pgaio_io_get_target_data(), PGAIO_RS_OK, pgstat_report_checksum_failures_in_db(), PgAioTargetData::rlocator, PgAioTargetData::smgr, and PgAioResult::status.

◆ shared_buffer_readv_stage()

static void shared_buffer_readv_stage ( PgAioHandle ioh,
uint8  cb_data 
)
static

Definition at line 7371 of file bufmgr.c.

7372{
7373 buffer_stage_common(ioh, false, false);
7374}

References buffer_stage_common().

◆ shared_buffer_write_error_callback()

static void shared_buffer_write_error_callback ( void *  arg)
static

Definition at line 6198 of file bufmgr.c.

6199{
6200 BufferDesc *bufHdr = (BufferDesc *) arg;
6201
6202 /* Buffer is pinned, so we can read the tag without locking the spinlock */
6203 if (bufHdr != NULL)
6204 errcontext("writing block %u of relation %s",
6205 bufHdr->tag.blockNum,
6207 BufTagGetForkNum(&bufHdr->tag)).str);
6208}

References arg, buftag::blockNum, BufTagGetForkNum(), BufTagGetRelFileLocator(), errcontext, relpathperm, and BufferDesc::tag.

Referenced by FlushBuffer().

◆ StartBufferIO()

bool StartBufferIO ( BufferDesc buf,
bool  forInput,
bool  nowait 
)

Definition at line 6043 of file bufmgr.c.

6044{
6045 uint32 buf_state;
6046
6048
6049 for (;;)
6050 {
6051 buf_state = LockBufHdr(buf);
6052
6053 if (!(buf_state & BM_IO_IN_PROGRESS))
6054 break;
6055 UnlockBufHdr(buf, buf_state);
6056 if (nowait)
6057 return false;
6058 WaitIO(buf);
6059 }
6060
6061 /* Once we get here, there is definitely no I/O active on this buffer */
6062
6063 /* Check if someone else already did the I/O */
6064 if (forInput ? (buf_state & BM_VALID) : !(buf_state & BM_DIRTY))
6065 {
6066 UnlockBufHdr(buf, buf_state);
6067 return false;
6068 }
6069
6070 buf_state |= BM_IO_IN_PROGRESS;
6071 UnlockBufHdr(buf, buf_state);
6072
6075
6076 return true;
6077}
static void ResourceOwnerRememberBufferIO(ResourceOwner owner, Buffer buffer)

References BM_DIRTY, BM_IO_IN_PROGRESS, BM_VALID, buf, BufferDescriptorGetBuffer(), CurrentResourceOwner, LockBufHdr(), ResourceOwnerEnlarge(), ResourceOwnerRememberBufferIO(), UnlockBufHdr(), and WaitIO().

Referenced by buffer_call_start_io(), ExtendBufferedRelShared(), FlushBuffer(), read_rel_block_ll(), ReadBuffersCanStartIOOnce(), and ZeroAndLockBuffer().

◆ StartReadBuffer()

bool StartReadBuffer ( ReadBuffersOperation operation,
Buffer buffer,
BlockNumber  blocknum,
int  flags 
)

Definition at line 1513 of file bufmgr.c.

1517{
1518 int nblocks = 1;
1519 bool result;
1520
1521 result = StartReadBuffersImpl(operation, buffer, blocknum, &nblocks, flags,
1522 false /* single block, no forwarding */ );
1523 Assert(nblocks == 1); /* single block can't be short */
1524
1525 return result;
1526}
static pg_attribute_always_inline bool StartReadBuffersImpl(ReadBuffersOperation *operation, Buffer *buffers, BlockNumber blockNum, int *nblocks, int flags, bool allow_forwarding)
Definition: bufmgr.c:1262

References Assert(), PrivateRefCountEntry::buffer, and StartReadBuffersImpl().

Referenced by read_stream_next_buffer(), and ReadBuffer_common().

◆ StartReadBuffers()

bool StartReadBuffers ( ReadBuffersOperation operation,
Buffer buffers,
BlockNumber  blockNum,
int *  nblocks,
int  flags 
)

Definition at line 1494 of file bufmgr.c.

1499{
1500 return StartReadBuffersImpl(operation, buffers, blockNum, nblocks, flags,
1501 true /* expect forwarded buffers */ );
1502}

References StartReadBuffersImpl().

Referenced by read_stream_start_pending_read().

◆ StartReadBuffersImpl()

static pg_attribute_always_inline bool StartReadBuffersImpl ( ReadBuffersOperation operation,
Buffer buffers,
BlockNumber  blockNum,
int *  nblocks,
int  flags,
bool  allow_forwarding 
)
static

Definition at line 1262 of file bufmgr.c.

1268{
1269 int actual_nblocks = *nblocks;
1270 int maxcombine = 0;
1271 bool did_start_io;
1272
1273 Assert(*nblocks == 1 || allow_forwarding);
1274 Assert(*nblocks > 0);
1275 Assert(*nblocks <= MAX_IO_COMBINE_LIMIT);
1276
1277 for (int i = 0; i < actual_nblocks; ++i)
1278 {
1279 bool found;
1280
1281 if (allow_forwarding && buffers[i] != InvalidBuffer)
1282 {
1283 BufferDesc *bufHdr;
1284
1285 /*
1286 * This is a buffer that was pinned by an earlier call to
1287 * StartReadBuffers(), but couldn't be handled in one operation at
1288 * that time. The operation was split, and the caller has passed
1289 * an already pinned buffer back to us to handle the rest of the
1290 * operation. It must continue at the expected block number.
1291 */
1292 Assert(BufferGetBlockNumber(buffers[i]) == blockNum + i);
1293
1294 /*
1295 * It might be an already valid buffer (a hit) that followed the
1296 * final contiguous block of an earlier I/O (a miss) marking the
1297 * end of it, or a buffer that some other backend has since made
1298 * valid by performing the I/O for us, in which case we can handle
1299 * it as a hit now. It is safe to check for a BM_VALID flag with
1300 * a relaxed load, because we got a fresh view of it while pinning
1301 * it in the previous call.
1302 *
1303 * On the other hand if we don't see BM_VALID yet, it must be an
1304 * I/O that was split by the previous call and we need to try to
1305 * start a new I/O from this block. We're also racing against any
1306 * other backend that might start the I/O or even manage to mark
1307 * it BM_VALID after this check, but StartBufferIO() will handle
1308 * those cases.
1309 */
1310 if (BufferIsLocal(buffers[i]))
1311 bufHdr = GetLocalBufferDescriptor(-buffers[i] - 1);
1312 else
1313 bufHdr = GetBufferDescriptor(buffers[i] - 1);
1315 found = pg_atomic_read_u32(&bufHdr->state) & BM_VALID;
1316 }
1317 else
1318 {
1319 buffers[i] = PinBufferForBlock(operation->rel,
1320 operation->smgr,
1321 operation->persistence,
1322 operation->forknum,
1323 blockNum + i,
1324 operation->strategy,
1325 &found);
1326 }
1327
1328 if (found)
1329 {
1330 /*
1331 * We have a hit. If it's the first block in the requested range,
1332 * we can return it immediately and report that WaitReadBuffers()
1333 * does not need to be called. If the initial value of *nblocks
1334 * was larger, the caller will have to call again for the rest.
1335 */
1336 if (i == 0)
1337 {
1338 *nblocks = 1;
1339
1340#ifdef USE_ASSERT_CHECKING
1341
1342 /*
1343 * Initialize enough of ReadBuffersOperation to make
1344 * CheckReadBuffersOperation() work. Outside of assertions
1345 * that's not necessary when no IO is issued.
1346 */
1347 operation->buffers = buffers;
1348 operation->blocknum = blockNum;
1349 operation->nblocks = 1;
1350 operation->nblocks_done = 1;
1351 CheckReadBuffersOperation(operation, true);
1352#endif
1353 return false;
1354 }
1355
1356 /*
1357 * Otherwise we already have an I/O to perform, but this block
1358 * can't be included as it is already valid. Split the I/O here.
1359 * There may or may not be more blocks requiring I/O after this
1360 * one, we haven't checked, but they can't be contiguous with this
1361 * one in the way. We'll leave this buffer pinned, forwarding it
1362 * to the next call, avoiding the need to unpin it here and re-pin
1363 * it in the next call.
1364 */
1365 actual_nblocks = i;
1366 break;
1367 }
1368 else
1369 {
1370 /*
1371 * Check how many blocks we can cover with the same IO. The smgr
1372 * implementation might e.g. be limited due to a segment boundary.
1373 */
1374 if (i == 0 && actual_nblocks > 1)
1375 {
1376 maxcombine = smgrmaxcombine(operation->smgr,
1377 operation->forknum,
1378 blockNum);
1379 if (unlikely(maxcombine < actual_nblocks))
1380 {
1381 elog(DEBUG2, "limiting nblocks at %u from %u to %u",
1382 blockNum, actual_nblocks, maxcombine);
1383 actual_nblocks = maxcombine;
1384 }
1385 }
1386 }
1387 }
1388 *nblocks = actual_nblocks;
1389
1390 /* Populate information needed for I/O. */
1391 operation->buffers = buffers;
1392 operation->blocknum = blockNum;
1393 operation->flags = flags;
1394 operation->nblocks = actual_nblocks;
1395 operation->nblocks_done = 0;
1396 pgaio_wref_clear(&operation->io_wref);
1397
1398 /*
1399 * When using AIO, start the IO in the background. If not, issue prefetch
1400 * requests if desired by the caller.
1401 *
1402 * The reason we have a dedicated path for IOMETHOD_SYNC here is to
1403 * de-risk the introduction of AIO somewhat. It's a large architectural
1404 * change, with lots of chances for unanticipated performance effects.
1405 *
1406 * Use of IOMETHOD_SYNC already leads to not actually performing IO
1407 * asynchronously, but without the check here we'd execute IO earlier than
1408 * we used to. Eventually this IOMETHOD_SYNC specific path should go away.
1409 */
1410 if (io_method != IOMETHOD_SYNC)
1411 {
1412 /*
1413 * Try to start IO asynchronously. It's possible that no IO needs to
1414 * be started, if another backend already performed the IO.
1415 *
1416 * Note that if an IO is started, it might not cover the entire
1417 * requested range, e.g. because an intermediary block has been read
1418 * in by another backend. In that case any "trailing" buffers we
1419 * already pinned above will be "forwarded" by read_stream.c to the
1420 * next call to StartReadBuffers().
1421 *
1422 * This is signalled to the caller by decrementing *nblocks *and*
1423 * reducing operation->nblocks. The latter is done here, but not below
1424 * WaitReadBuffers(), as in WaitReadBuffers() we can't "shorten" the
1425 * overall read size anymore, we need to retry until done in its
1426 * entirety or until failed.
1427 */
1428 did_start_io = AsyncReadBuffers(operation, nblocks);
1429
1430 operation->nblocks = *nblocks;
1431 }
1432 else
1433 {
1434 operation->flags |= READ_BUFFERS_SYNCHRONOUSLY;
1435
1436 if (flags & READ_BUFFERS_ISSUE_ADVICE)
1437 {
1438 /*
1439 * In theory we should only do this if PinBufferForBlock() had to
1440 * allocate new buffers above. That way, if two calls to
1441 * StartReadBuffers() were made for the same blocks before
1442 * WaitReadBuffers(), only the first would issue the advice.
1443 * That'd be a better simulation of true asynchronous I/O, which
1444 * would only start the I/O once, but isn't done here for
1445 * simplicity.
1446 */
1447 smgrprefetch(operation->smgr,
1448 operation->forknum,
1449 blockNum,
1450 actual_nblocks);
1451 }
1452
1453 /*
1454 * Indicate that WaitReadBuffers() should be called. WaitReadBuffers()
1455 * will initiate the necessary IO.
1456 */
1457 did_start_io = true;
1458 }
1459
1460 CheckReadBuffersOperation(operation, !did_start_io);
1461
1462 return did_start_io;
1463}
int io_method
Definition: aio.c:74
@ IOMETHOD_SYNC
Definition: aio.h:34
static void CheckReadBuffersOperation(ReadBuffersOperation *operation, bool is_complete)
Definition: bufmgr.c:1532
static bool AsyncReadBuffers(ReadBuffersOperation *operation, int *nblocks_progress)
Definition: bufmgr.c:1769
#define READ_BUFFERS_ISSUE_ADVICE
Definition: bufmgr.h:114
uint32 smgrmaxcombine(SMgrRelation reln, ForkNumber forknum, BlockNumber blocknum)
Definition: smgr.c:697

References Assert(), AsyncReadBuffers(), ReadBuffersOperation::blocknum, BM_TAG_VALID, BM_VALID, BufferGetBlockNumber(), BufferIsLocal, ReadBuffersOperation::buffers, CheckReadBuffersOperation(), DEBUG2, elog, ReadBuffersOperation::flags, ReadBuffersOperation::forknum, GetBufferDescriptor(), GetLocalBufferDescriptor(), i, InvalidBuffer, io_method, ReadBuffersOperation::io_wref, IOMETHOD_SYNC, MAX_IO_COMBINE_LIMIT, ReadBuffersOperation::nblocks, ReadBuffersOperation::nblocks_done, ReadBuffersOperation::persistence, pg_atomic_read_u32(), pgaio_wref_clear(), PinBufferForBlock(), READ_BUFFERS_ISSUE_ADVICE, READ_BUFFERS_SYNCHRONOUSLY, ReadBuffersOperation::rel, ReadBuffersOperation::smgr, smgrmaxcombine(), smgrprefetch(), BufferDesc::state, ReadBuffersOperation::strategy, and unlikely.

Referenced by StartReadBuffer(), and StartReadBuffers().

◆ SyncOneBuffer()

static int SyncOneBuffer ( int  buf_id,
bool  skip_recently_used,
WritebackContext wb_context 
)
static

Definition at line 3923 of file bufmgr.c.

3924{
3925 BufferDesc *bufHdr = GetBufferDescriptor(buf_id);
3926 int result = 0;
3927 uint32 buf_state;
3928 BufferTag tag;
3929
3930 /* Make sure we can handle the pin */
3933
3934 /*
3935 * Check whether buffer needs writing.
3936 *
3937 * We can make this check without taking the buffer content lock so long
3938 * as we mark pages dirty in access methods *before* logging changes with
3939 * XLogInsert(): if someone marks the buffer dirty just after our check we
3940 * don't worry because our checkpoint.redo points before log record for
3941 * upcoming changes and so we are not required to write such dirty buffer.
3942 */
3943 buf_state = LockBufHdr(bufHdr);
3944
3945 if (BUF_STATE_GET_REFCOUNT(buf_state) == 0 &&
3946 BUF_STATE_GET_USAGECOUNT(buf_state) == 0)
3947 {
3948 result |= BUF_REUSABLE;
3949 }
3950 else if (skip_recently_used)
3951 {
3952 /* Caller told us not to write recently-used buffers */
3953 UnlockBufHdr(bufHdr, buf_state);
3954 return result;
3955 }
3956
3957 if (!(buf_state & BM_VALID) || !(buf_state & BM_DIRTY))
3958 {
3959 /* It's clean, so nothing to do */
3960 UnlockBufHdr(bufHdr, buf_state);
3961 return result;
3962 }
3963
3964 /*
3965 * Pin it, share-lock it, write it. (FlushBuffer will do nothing if the
3966 * buffer is clean by the time we've locked it.)
3967 */
3968 PinBuffer_Locked(bufHdr);
3970
3972
3974
3975 tag = bufHdr->tag;
3976
3977 UnpinBuffer(bufHdr);
3978
3979 /*
3980 * SyncOneBuffer() is only called by checkpointer and bgwriter, so
3981 * IOContext will always be IOCONTEXT_NORMAL.
3982 */
3984
3985 return result | BUF_WRITTEN;
3986}

References BM_DIRTY, BM_VALID, BUF_REUSABLE, BUF_STATE_GET_REFCOUNT, BUF_STATE_GET_USAGECOUNT, BUF_WRITTEN, BufferDescriptorGetContentLock(), CurrentResourceOwner, FlushBuffer(), GetBufferDescriptor(), IOCONTEXT_NORMAL, IOOBJECT_RELATION, LockBufHdr(), LW_SHARED, LWLockAcquire(), LWLockRelease(), PinBuffer_Locked(), ReservePrivateRefCountEntry(), ResourceOwnerEnlarge(), ScheduleBufferTagForWriteback(), BufferDesc::tag, UnlockBufHdr(), and UnpinBuffer().

Referenced by BgBufferSync(), and BufferSync().

◆ TerminateBufferIO()

void TerminateBufferIO ( BufferDesc buf,
bool  clear_dirty,
uint32  set_flag_bits,
bool  forget_owner,
bool  release_aio 
)

Definition at line 6100 of file bufmgr.c.

6102{
6103 uint32 buf_state;
6104
6105 buf_state = LockBufHdr(buf);
6106
6107 Assert(buf_state & BM_IO_IN_PROGRESS);
6108 buf_state &= ~BM_IO_IN_PROGRESS;
6109
6110 /* Clear earlier errors, if this IO failed, it'll be marked again */
6111 buf_state &= ~BM_IO_ERROR;
6112
6113 if (clear_dirty && !(buf_state & BM_JUST_DIRTIED))
6114 buf_state &= ~(BM_DIRTY | BM_CHECKPOINT_NEEDED);
6115
6116 if (release_aio)
6117 {
6118 /* release ownership by the AIO subsystem */
6119 Assert(BUF_STATE_GET_REFCOUNT(buf_state) > 0);
6120 buf_state -= BUF_REFCOUNT_ONE;
6121 pgaio_wref_clear(&buf->io_wref);
6122 }
6123
6124 buf_state |= set_flag_bits;
6125 UnlockBufHdr(buf, buf_state);
6126
6127 if (forget_owner)
6130
6132
6133 /*
6134 * Support LockBufferForCleanup()
6135 *
6136 * We may have just released the last pin other than the waiter's. In most
6137 * cases, this backend holds another pin on the buffer. But, if, for
6138 * example, this backend is completing an IO issued by another backend, it
6139 * may be time to wake the waiter.
6140 */
6141 if (release_aio && (buf_state & BM_PIN_COUNT_WAITER))
6143}
static ConditionVariable * BufferDescriptorGetIOCV(const BufferDesc *bdesc)
static void WakePinCountWaiter(BufferDesc *buf)
Definition: bufmgr.c:3229
void ConditionVariableBroadcast(ConditionVariable *cv)

References Assert(), BM_CHECKPOINT_NEEDED, BM_DIRTY, BM_IO_IN_PROGRESS, BM_JUST_DIRTIED, BM_PIN_COUNT_WAITER, buf, BUF_REFCOUNT_ONE, BUF_STATE_GET_REFCOUNT, BufferDescriptorGetBuffer(), BufferDescriptorGetIOCV(), ConditionVariableBroadcast(), CurrentResourceOwner, LockBufHdr(), pgaio_wref_clear(), ResourceOwnerForgetBufferIO(), UnlockBufHdr(), and WakePinCountWaiter().

Referenced by AbortBufferIO(), buffer_call_terminate_io(), buffer_readv_complete_one(), ExtendBufferedRelShared(), FlushBuffer(), and ZeroAndLockBuffer().

◆ ts_ckpt_progress_comparator()

static int ts_ckpt_progress_comparator ( Datum  a,
Datum  b,
void *  arg 
)
static

Definition at line 6376 of file bufmgr.c.

6377{
6379 CkptTsStatus *sb = (CkptTsStatus *) b;
6380
6381 /* we want a min-heap, so return 1 for the a < b */
6382 if (sa->progress < sb->progress)
6383 return 1;
6384 else if (sa->progress == sb->progress)
6385 return 0;
6386 else
6387 return -1;
6388}

References a, b, and CkptTsStatus::progress.

Referenced by BufferSync().

◆ UnlockBuffers()

void UnlockBuffers ( void  )

Definition at line 5577 of file bufmgr.c.

5578{
5580
5581 if (buf)
5582 {
5583 uint32 buf_state;
5584
5585 buf_state = LockBufHdr(buf);
5586
5587 /*
5588 * Don't complain if flag bit not set; it could have been reset but we
5589 * got a cancel/die interrupt before getting the signal.
5590 */
5591 if ((buf_state & BM_PIN_COUNT_WAITER) != 0 &&
5592 buf->wait_backend_pgprocno == MyProcNumber)
5593 buf_state &= ~BM_PIN_COUNT_WAITER;
5594
5595 UnlockBufHdr(buf, buf_state);
5596
5597 PinCountWaitBuf = NULL;
5598 }
5599}

References BM_PIN_COUNT_WAITER, buf, LockBufHdr(), MyProcNumber, PinCountWaitBuf, and UnlockBufHdr().

Referenced by AbortSubTransaction(), AbortTransaction(), AtProcExit_Buffers(), AutoVacLauncherMain(), BackgroundWriterMain(), CheckpointerMain(), and WalWriterMain().

◆ UnlockReleaseBuffer()

void UnlockReleaseBuffer ( Buffer  buffer)

Definition at line 5388 of file bufmgr.c.

5389{
5391 ReleaseBuffer(buffer);
5392}

References PrivateRefCountEntry::buffer, BUFFER_LOCK_UNLOCK, LockBuffer(), and ReleaseBuffer().

Referenced by _bt_clear_incomplete_split(), _bt_restore_meta(), _hash_relbuf(), allocNewBuffer(), AlterSequence(), blbulkdelete(), blgetbitmap(), blinsert(), BloomInitMetapage(), blvacuumcleanup(), brin_doinsert(), brin_doupdate(), brin_evacuate_page(), brin_getinsertbuffer(), brin_xlog_createidx(), brin_xlog_desummarize_page(), brin_xlog_insert_update(), brin_xlog_revmap_extend(), brin_xlog_samepage_update(), brin_xlog_update(), brinbuild(), brinbuildempty(), brinGetStats(), brinRevmapDesummarizeRange(), bt_metap(), bt_multi_page_stats(), bt_page_items_internal(), bt_page_stats_internal(), bt_recheck_sibling_links(), btree_xlog_dedup(), btree_xlog_delete(), btree_xlog_insert(), btree_xlog_mark_page_halfdead(), btree_xlog_newroot(), btree_xlog_split(), btree_xlog_unlink_page(), btree_xlog_vacuum(), collect_corrupt_items(), collect_visibility_data(), count_nondeletable_pages(), createPostingTree(), do_setval(), doPickSplit(), entryLoadMoreItems(), fill_seq_fork_with_data(), flushCachedPage(), FreeSpaceMapPrepareTruncateRel(), fsm_search(), fsm_set_and_search(), generic_redo(), gin_refind_parent(), ginbuild(), ginbuildempty(), ginbulkdelete(), ginGetStats(), ginHeapTupleFastInsert(), ginInsertCleanup(), ginPlaceToPage(), ginRedoClearIncompleteSplit(), ginRedoCreatePTree(), ginRedoDeleteListPages(), ginRedoDeletePage(), ginRedoInsert(), ginRedoInsertListPage(), ginRedoSplit(), ginRedoUpdateMetapage(), ginRedoVacuumDataLeafPage(), ginRedoVacuumPage(), ginScanToDelete(), ginStepRight(), ginUpdateStats(), ginvacuumcleanup(), ginVacuumPostingTree(), ginVacuumPostingTreeLeaves(), gistbufferinginserttuples(), gistbuild(), gistbuildempty(), gistdoinsert(), gistFindCorrectParent(), gistFindPath(), gistGetMaxLevel(), gistinserttuples(), gistkillitems(), gistplacetopage(), gistProcessItup(), gistRedoClearFollowRight(), gistRedoDeleteRecord(), gistRedoPageDelete(), gistRedoPageSplitRecord(), gistRedoPageUpdateRecord(), gistScanPage(), gistvacuum_delete_empty_pages(), gistvacuumpage(), hash_xlog_add_ovfl_page(), hash_xlog_delete(), hash_xlog_init_bitmap_page(), hash_xlog_init_meta_page(), hash_xlog_insert(), hash_xlog_move_page_contents(), hash_xlog_split_allocate_page(), hash_xlog_split_cleanup(), hash_xlog_split_complete(), hash_xlog_split_page(), hash_xlog_squeeze_page(), hash_xlog_update_meta_page(), hash_xlog_vacuum_one_page(), heap_delete(), heap_finish_speculative(), heap_force_common(), heap_get_latest_tid(), heap_index_delete_tuples(), heap_insert(), heap_lock_updated_tuple_rec(), heap_multi_insert(), heap_update(), heap_xlog_confirm(), heap_xlog_delete(), heap_xlog_inplace(), heap_xlog_insert(), heap_xlog_lock(), heap_xlog_lock_updated(), heap_xlog_multi_insert(), heap_xlog_prune_freeze(), heap_xlog_update(), heap_xlog_visible(), heapam_scan_analyze_next_tuple(), initBloomState(), lazy_scan_heap(), lazy_scan_new_or_empty(), lazy_vacuum_heap_rel(), log_newpage_range(), moveLeafs(), nextval_internal(), palloc_btree_page(), pg_get_sequence_data(), pg_sequence_last_value(), pg_visibility(), pgstat_gist_page(), pgstat_heap(), pgstatginindex_internal(), pgstathashindex(), RelationCopyStorageUsingBuffer(), RelationGetBufferForTuple(), ResetSequence(), revmap_physical_extend(), scanGetCandidate(), scanPendingInsert(), scanPostingTree(), ScanSourceDatabasePgClass(), seq_redo(), SequenceChangePersistence(), shiftList(), spgAddNodeAction(), spgbuild(), spgdoinsert(), spgGetCache(), SpGistGetBuffer(), SpGistUpdateMetaPage(), spgMatchNodeAction(), spgprocesspending(), spgRedoAddLeaf(), spgRedoAddNode(), spgRedoMoveLeafs(), spgRedoPickSplit(), spgRedoSplitTuple(), spgRedoVacuumLeaf(), spgRedoVacuumRedirect(), spgRedoVacuumRoot(), spgSplitNodeAction(), spgvacuumpage(), spgWalk(), statapprox_heap(), verify_heapam(), verifyBackupPageConsistency(), visibilitymap_prepare_truncate(), writeListPage(), xlog_redo(), and XLogRecordPageWithFreeSpace().

◆ UnpinBuffer()

◆ UnpinBufferNoOwner()

static void UnpinBufferNoOwner ( BufferDesc buf)
static

Definition at line 3273 of file bufmgr.c.

3274{
3277
3279
3280 /* not moving as we're likely deleting it soon anyway */
3281 ref = GetPrivateRefCountEntry(b, false);
3282 Assert(ref != NULL);
3283 Assert(ref->refcount > 0);
3284 ref->refcount--;
3285 if (ref->refcount == 0)
3286 {
3287 uint32 buf_state;
3288 uint32 old_buf_state;
3289
3290 /*
3291 * Mark buffer non-accessible to Valgrind.
3292 *
3293 * Note that the buffer may have already been marked non-accessible
3294 * within access method code that enforces that buffers are only
3295 * accessed while a buffer lock is held.
3296 */
3298
3299 /* I'd better not still hold the buffer content lock */
3301
3302 /*
3303 * Decrement the shared reference count.
3304 *
3305 * Since buffer spinlock holder can update status using just write,
3306 * it's not safe to use atomic decrement here; thus use a CAS loop.
3307 */
3308 old_buf_state = pg_atomic_read_u32(&buf->state);
3309 for (;;)
3310 {
3311 if (old_buf_state & BM_LOCKED)
3312 old_buf_state = WaitBufHdrUnlocked(buf);
3313
3314 buf_state = old_buf_state;
3315
3316 buf_state -= BUF_REFCOUNT_ONE;
3317
3318 if (pg_atomic_compare_exchange_u32(&buf->state, &old_buf_state,
3319 buf_state))
3320 break;
3321 }
3322
3323 /* Support LockBufferForCleanup() */
3324 if (buf_state & BM_PIN_COUNT_WAITER)
3326
3328 }
3329}
static void ForgetPrivateRefCountEntry(PrivateRefCountEntry *ref)
Definition: bufmgr.c:448

References Assert(), b, BM_LOCKED, BM_PIN_COUNT_WAITER, buf, BUF_REFCOUNT_ONE, BufferDescriptorGetBuffer(), BufferDescriptorGetContentLock(), BufferIsLocal, BufHdrGetBlock, ForgetPrivateRefCountEntry(), GetPrivateRefCountEntry(), LWLockHeldByMe(), pg_atomic_compare_exchange_u32(), pg_atomic_read_u32(), PrivateRefCountEntry::refcount, VALGRIND_MAKE_MEM_NOACCESS, WaitBufHdrUnlocked(), and WakePinCountWaiter().

Referenced by ResOwnerReleaseBufferPin(), and UnpinBuffer().

◆ WaitBufHdrUnlocked()

static uint32 WaitBufHdrUnlocked ( BufferDesc buf)
static

Definition at line 6287 of file bufmgr.c.

6288{
6289 SpinDelayStatus delayStatus;
6290 uint32 buf_state;
6291
6292 init_local_spin_delay(&delayStatus);
6293
6294 buf_state = pg_atomic_read_u32(&buf->state);
6295
6296 while (buf_state & BM_LOCKED)
6297 {
6298 perform_spin_delay(&delayStatus);
6299 buf_state = pg_atomic_read_u32(&buf->state);
6300 }
6301
6302 finish_spin_delay(&delayStatus);
6303
6304 return buf_state;
6305}

References BM_LOCKED, buf, finish_spin_delay(), init_local_spin_delay, perform_spin_delay(), and pg_atomic_read_u32().

Referenced by MarkBufferDirty(), PinBuffer(), and UnpinBufferNoOwner().

◆ WaitIO()

static void WaitIO ( BufferDesc buf)
static

Definition at line 5964 of file bufmgr.c.

5965{
5967
5969 for (;;)
5970 {
5971 uint32 buf_state;
5972 PgAioWaitRef iow;
5973
5974 /*
5975 * It may not be necessary to acquire the spinlock to check the flag
5976 * here, but since this test is essential for correctness, we'd better
5977 * play it safe.
5978 */
5979 buf_state = LockBufHdr(buf);
5980
5981 /*
5982 * Copy the wait reference while holding the spinlock. This protects
5983 * against a concurrent TerminateBufferIO() in another backend from
5984 * clearing the wref while it's being read.
5985 */
5986 iow = buf->io_wref;
5987 UnlockBufHdr(buf, buf_state);
5988
5989 /* no IO in progress, we don't need to wait */
5990 if (!(buf_state & BM_IO_IN_PROGRESS))
5991 break;
5992
5993 /*
5994 * The buffer has asynchronous IO in progress, wait for it to
5995 * complete.
5996 */
5997 if (pgaio_wref_valid(&iow))
5998 {
5999 pgaio_wref_wait(&iow);
6000
6001 /*
6002 * The AIO subsystem internally uses condition variables and thus
6003 * might remove this backend from the BufferDesc's CV. While that
6004 * wouldn't cause a correctness issue (the first CV sleep just
6005 * immediately returns if not already registered), it seems worth
6006 * avoiding unnecessary loop iterations, given that we take care
6007 * to do so at the start of the function.
6008 */
6010 continue;
6011 }
6012
6013 /* wait on BufferDesc->cv, e.g. for concurrent synchronous IO */
6014 ConditionVariableSleep(cv, WAIT_EVENT_BUFFER_IO);
6015 }
6017}
void pgaio_wref_wait(PgAioWaitRef *iow)
Definition: aio.c:971
bool ConditionVariableCancelSleep(void)
void ConditionVariablePrepareToSleep(ConditionVariable *cv)
void ConditionVariableSleep(ConditionVariable *cv, uint32 wait_event_info)

References BM_IO_IN_PROGRESS, buf, BufferDescriptorGetIOCV(), ConditionVariableCancelSleep(), ConditionVariablePrepareToSleep(), ConditionVariableSleep(), LockBufHdr(), pgaio_wref_valid(), pgaio_wref_wait(), and UnlockBufHdr().

Referenced by InvalidateBuffer(), and StartBufferIO().

◆ WaitReadBuffers()

void WaitReadBuffers ( ReadBuffersOperation operation)

Definition at line 1637 of file bufmgr.c.

1638{
1639 PgAioReturn *aio_ret = &operation->io_return;
1640 IOContext io_context;
1641 IOObject io_object;
1642
1643 if (operation->persistence == RELPERSISTENCE_TEMP)
1644 {
1645 io_context = IOCONTEXT_NORMAL;
1646 io_object = IOOBJECT_TEMP_RELATION;
1647 }
1648 else
1649 {
1650 io_context = IOContextForStrategy(operation->strategy);
1651 io_object = IOOBJECT_RELATION;
1652 }
1653
1654 /*
1655 * If we get here without an IO operation having been issued, the
1656 * io_method == IOMETHOD_SYNC path must have been used. Otherwise the
1657 * caller should not have called WaitReadBuffers().
1658 *
1659 * In the case of IOMETHOD_SYNC, we start - as we used to before the
1660 * introducing of AIO - the IO in WaitReadBuffers(). This is done as part
1661 * of the retry logic below, no extra code is required.
1662 *
1663 * This path is expected to eventually go away.
1664 */
1665 if (!pgaio_wref_valid(&operation->io_wref) && io_method != IOMETHOD_SYNC)
1666 elog(ERROR, "waiting for read operation that didn't read");
1667
1668 /*
1669 * To handle partial reads, and IOMETHOD_SYNC, we re-issue IO until we're
1670 * done. We may need multiple retries, not just because we could get
1671 * multiple partial reads, but also because some of the remaining
1672 * to-be-read buffers may have been read in by other backends, limiting
1673 * the IO size.
1674 */
1675 while (true)
1676 {
1677 int ignored_nblocks_progress;
1678
1679 CheckReadBuffersOperation(operation, false);
1680
1681 /*
1682 * If there is an IO associated with the operation, we may need to
1683 * wait for it.
1684 */
1685 if (pgaio_wref_valid(&operation->io_wref))
1686 {
1687 /*
1688 * Track the time spent waiting for the IO to complete. As
1689 * tracking a wait even if we don't actually need to wait
1690 *
1691 * a) is not cheap, due to the timestamping overhead
1692 *
1693 * b) reports some time as waiting, even if we never waited
1694 *
1695 * we first check if we already know the IO is complete.
1696 */
1697 if (aio_ret->result.status == PGAIO_RS_UNKNOWN &&
1698 !pgaio_wref_check_done(&operation->io_wref))
1699 {
1701
1702 pgaio_wref_wait(&operation->io_wref);
1703
1704 /*
1705 * The IO operation itself was already counted earlier, in
1706 * AsyncReadBuffers(), this just accounts for the wait time.
1707 */
1708 pgstat_count_io_op_time(io_object, io_context, IOOP_READ,
1709 io_start, 0, 0);
1710 }
1711 else
1712 {
1713 Assert(pgaio_wref_check_done(&operation->io_wref));
1714 }
1715
1716 /*
1717 * We now are sure the IO completed. Check the results. This
1718 * includes reporting on errors if there were any.
1719 */
1720 ProcessReadBuffersResult(operation);
1721 }
1722
1723 /*
1724 * Most of the time, the one IO we already started, will read in
1725 * everything. But we need to deal with partial reads and buffers not
1726 * needing IO anymore.
1727 */
1728 if (operation->nblocks_done == operation->nblocks)
1729 break;
1730
1732
1733 /*
1734 * This may only complete the IO partially, either because some
1735 * buffers were already valid, or because of a partial read.
1736 *
1737 * NB: In contrast to after the AsyncReadBuffers() call in
1738 * StartReadBuffers(), we do *not* reduce
1739 * ReadBuffersOperation->nblocks here, callers expect the full
1740 * operation to be completed at this point (as more operations may
1741 * have been queued).
1742 */
1743 AsyncReadBuffers(operation, &ignored_nblocks_progress);
1744 }
1745
1746 CheckReadBuffersOperation(operation, true);
1747
1748 /* NB: READ_DONE tracepoint was already executed in completion callback */
1749}
bool pgaio_wref_check_done(PgAioWaitRef *iow)
Definition: aio.c:985
static void ProcessReadBuffersResult(ReadBuffersOperation *operation)
Definition: bufmgr.c:1598

References Assert(), AsyncReadBuffers(), CHECK_FOR_INTERRUPTS, CheckReadBuffersOperation(), elog, ERROR, io_method, ReadBuffersOperation::io_return, ReadBuffersOperation::io_wref, IOCONTEXT_NORMAL, IOContextForStrategy(), IOMETHOD_SYNC, IOOBJECT_RELATION, IOOBJECT_TEMP_RELATION, IOOP_READ, ReadBuffersOperation::nblocks, ReadBuffersOperation::nblocks_done, ReadBuffersOperation::persistence, PGAIO_RS_UNKNOWN, pgaio_wref_check_done(), pgaio_wref_valid(), pgaio_wref_wait(), pgstat_count_io_op_time(), pgstat_prepare_io_time(), ProcessReadBuffersResult(), PgAioReturn::result, PgAioResult::status, ReadBuffersOperation::strategy, and track_io_timing.

Referenced by read_stream_next_buffer(), and ReadBuffer_common().

◆ WakePinCountWaiter()

static void WakePinCountWaiter ( BufferDesc buf)
static

Definition at line 3229 of file bufmgr.c.

3230{
3231 /*
3232 * Acquire the buffer header lock, re-check that there's a waiter. Another
3233 * backend could have unpinned this buffer, and already woken up the
3234 * waiter.
3235 *
3236 * There's no danger of the buffer being replaced after we unpinned it
3237 * above, as it's pinned by the waiter. The waiter removes
3238 * BM_PIN_COUNT_WAITER if it stops waiting for a reason other than this
3239 * backend waking it up.
3240 */
3241 uint32 buf_state = LockBufHdr(buf);
3242
3243 if ((buf_state & BM_PIN_COUNT_WAITER) &&
3244 BUF_STATE_GET_REFCOUNT(buf_state) == 1)
3245 {
3246 /* we just released the last pin other than the waiter's */
3247 int wait_backend_pgprocno = buf->wait_backend_pgprocno;
3248
3249 buf_state &= ~BM_PIN_COUNT_WAITER;
3250 UnlockBufHdr(buf, buf_state);
3251 ProcSendSignal(wait_backend_pgprocno);
3252 }
3253 else
3254 UnlockBufHdr(buf, buf_state);
3255}
void ProcSendSignal(ProcNumber procNumber)
Definition: proc.c:1987

References BM_PIN_COUNT_WAITER, buf, BUF_STATE_GET_REFCOUNT, LockBufHdr(), ProcSendSignal(), and UnlockBufHdr().

Referenced by TerminateBufferIO(), and UnpinBufferNoOwner().

◆ WritebackContextInit()

void WritebackContextInit ( WritebackContext context,
int *  max_pending 
)

Definition at line 6399 of file bufmgr.c.

6400{
6401 Assert(*max_pending <= WRITEBACK_MAX_PENDING_FLUSHES);
6402
6403 context->max_pending = max_pending;
6404 context->nr_pending = 0;
6405}

References Assert(), WritebackContext::max_pending, WritebackContext::nr_pending, and WRITEBACK_MAX_PENDING_FLUSHES.

Referenced by BackgroundWriterMain(), BufferManagerShmemInit(), and BufferSync().

◆ ZeroAndLockBuffer()

static void ZeroAndLockBuffer ( Buffer  buffer,
ReadBufferMode  mode,
bool  already_valid 
)
static

Definition at line 1031 of file bufmgr.c.

1032{
1033 BufferDesc *bufHdr;
1034 bool need_to_zero;
1035 bool isLocalBuf = BufferIsLocal(buffer);
1036
1038
1039 if (already_valid)
1040 {
1041 /*
1042 * If the caller already knew the buffer was valid, we can skip some
1043 * header interaction. The caller just wants to lock the buffer.
1044 */
1045 need_to_zero = false;
1046 }
1047 else if (isLocalBuf)
1048 {
1049 /* Simple case for non-shared buffers. */
1050 bufHdr = GetLocalBufferDescriptor(-buffer - 1);
1051 need_to_zero = StartLocalBufferIO(bufHdr, true, false);
1052 }
1053 else
1054 {
1055 /*
1056 * Take BM_IO_IN_PROGRESS, or discover that BM_VALID has been set
1057 * concurrently. Even though we aren't doing I/O, that ensures that
1058 * we don't zero a page that someone else has pinned. An exclusive
1059 * content lock wouldn't be enough, because readers are allowed to
1060 * drop the content lock after determining that a tuple is visible
1061 * (see buffer access rules in README).
1062 */
1063 bufHdr = GetBufferDescriptor(buffer - 1);
1064 need_to_zero = StartBufferIO(bufHdr, true, false);
1065 }
1066
1067 if (need_to_zero)
1068 {
1069 memset(BufferGetPage(buffer), 0, BLCKSZ);
1070
1071 /*
1072 * Grab the buffer content lock before marking the page as valid, to
1073 * make sure that no other backend sees the zeroed page before the
1074 * caller has had a chance to initialize it.
1075 *
1076 * Since no-one else can be looking at the page contents yet, there is
1077 * no difference between an exclusive lock and a cleanup-strength
1078 * lock. (Note that we cannot use LockBuffer() or
1079 * LockBufferForCleanup() here, because they assert that the buffer is
1080 * already valid.)
1081 */
1082 if (!isLocalBuf)
1084
1085 /* Set BM_VALID, terminate IO, and wake up any waiters */
1086 if (isLocalBuf)
1087 TerminateLocalBufferIO(bufHdr, false, BM_VALID, false);
1088 else
1089 TerminateBufferIO(bufHdr, false, BM_VALID, true, false);
1090 }
1091 else if (!isLocalBuf)
1092 {
1093 /*
1094 * The buffer is valid, so we can't zero it. The caller still expects
1095 * the page to be locked on return.
1096 */
1097 if (mode == RBM_ZERO_AND_LOCK)
1099 else
1100 LockBufferForCleanup(buffer);
1101 }
1102}
void LockBufferForCleanup(Buffer buffer)
Definition: bufmgr.c:5685

References Assert(), BM_VALID, PrivateRefCountEntry::buffer, BUFFER_LOCK_EXCLUSIVE, BufferDescriptorGetContentLock(), BufferGetPage(), BufferIsLocal, GetBufferDescriptor(), GetLocalBufferDescriptor(), LockBuffer(), LockBufferForCleanup(), LW_EXCLUSIVE, LWLockAcquire(), mode, RBM_ZERO_AND_CLEANUP_LOCK, RBM_ZERO_AND_LOCK, StartBufferIO(), StartLocalBufferIO(), TerminateBufferIO(), and TerminateLocalBufferIO().

Referenced by ReadBuffer_common().

Variable Documentation

◆ aio_local_buffer_readv_cb

const PgAioHandleCallbacks aio_local_buffer_readv_cb
Initial value:
= {
.complete_local = local_buffer_readv_complete,
}
static PgAioResult local_buffer_readv_complete(PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
Definition: bufmgr.c:7428
static void local_buffer_readv_stage(PgAioHandle *ioh, uint8 cb_data)
Definition: bufmgr.c:7422
static void buffer_readv_report(PgAioResult result, const PgAioTargetData *td, int elevel)
Definition: bufmgr.c:7278

Definition at line 7444 of file bufmgr.c.

◆ aio_shared_buffer_readv_cb

const PgAioHandleCallbacks aio_shared_buffer_readv_cb
Initial value:
= {
.complete_shared = shared_buffer_readv_complete,
}
static PgAioResult shared_buffer_readv_complete_local(PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
Definition: bufmgr.c:7391
static void shared_buffer_readv_stage(PgAioHandle *ioh, uint8 cb_data)
Definition: bufmgr.c:7371
static PgAioResult shared_buffer_readv_complete(PgAioHandle *ioh, PgAioResult prior_result, uint8 cb_data)
Definition: bufmgr.c:7377

Definition at line 7435 of file bufmgr.c.

◆ backend_flush_after

int backend_flush_after = DEFAULT_BACKEND_FLUSH_AFTER

Definition at line 180 of file bufmgr.c.

Referenced by BufferManagerShmemInit().

◆ bgwriter_flush_after

int bgwriter_flush_after = DEFAULT_BGWRITER_FLUSH_AFTER

Definition at line 179 of file bufmgr.c.

Referenced by BackgroundWriterMain().

◆ bgwriter_lru_maxpages

int bgwriter_lru_maxpages = 100

Definition at line 145 of file bufmgr.c.

Referenced by BgBufferSync().

◆ bgwriter_lru_multiplier

double bgwriter_lru_multiplier = 2.0

Definition at line 146 of file bufmgr.c.

Referenced by BgBufferSync().

◆ buffer_io_resowner_desc

const ResourceOwnerDesc buffer_io_resowner_desc
Initial value:
=
{
.name = "buffer io",
.release_priority = RELEASE_PRIO_BUFFER_IOS,
.ReleaseResource = ResOwnerReleaseBufferIO,
.DebugPrint = ResOwnerPrintBufferIO
}
static void ResOwnerReleaseBufferIO(Datum res)
Definition: bufmgr.c:6544
static char * ResOwnerPrintBufferIO(Datum res)
Definition: bufmgr.c:6552
#define RELEASE_PRIO_BUFFER_IOS
Definition: resowner.h:62
@ RESOURCE_RELEASE_BEFORE_LOCKS
Definition: resowner.h:54

Definition at line 235 of file bufmgr.c.

Referenced by ResourceOwnerForgetBufferIO(), and ResourceOwnerRememberBufferIO().

◆ buffer_pin_resowner_desc

const ResourceOwnerDesc buffer_pin_resowner_desc
Initial value:
=
{
.name = "buffer pin",
.release_priority = RELEASE_PRIO_BUFFER_PINS,
.ReleaseResource = ResOwnerReleaseBufferPin,
.DebugPrint = ResOwnerPrintBufferPin
}
static char * ResOwnerPrintBufferPin(Datum res)
Definition: bufmgr.c:6575
static void ResOwnerReleaseBufferPin(Datum res)
Definition: bufmgr.c:6560
#define RELEASE_PRIO_BUFFER_PINS
Definition: resowner.h:63

Definition at line 244 of file bufmgr.c.

Referenced by ResourceOwnerForgetBuffer(), and ResourceOwnerRememberBuffer().

◆ checkpoint_flush_after

int checkpoint_flush_after = DEFAULT_CHECKPOINT_FLUSH_AFTER

Definition at line 178 of file bufmgr.c.

Referenced by BufferSync().

◆ effective_io_concurrency

◆ io_combine_limit

◆ io_combine_limit_guc

int io_combine_limit_guc = DEFAULT_IO_COMBINE_LIMIT

Definition at line 171 of file bufmgr.c.

Referenced by assign_io_max_combine_limit().

◆ io_max_combine_limit

◆ maintenance_io_concurrency

◆ MaxProportionalPins

uint32 MaxProportionalPins
static

Definition at line 221 of file bufmgr.c.

Referenced by GetAdditionalPinLimit(), GetPinLimit(), and InitBufferManagerAccess().

◆ PinCountWaitBuf

BufferDesc* PinCountWaitBuf = NULL
static

Definition at line 183 of file bufmgr.c.

Referenced by LockBufferForCleanup(), and UnlockBuffers().

◆ PrivateRefCountArray

◆ PrivateRefCountClock

uint32 PrivateRefCountClock = 0
static

Definition at line 218 of file bufmgr.c.

Referenced by ReservePrivateRefCountEntry().

◆ PrivateRefCountHash

◆ PrivateRefCountOverflowed

◆ ReservedRefCountEntry

◆ track_io_timing

◆ zero_damaged_pages

bool zero_damaged_pages = false

Definition at line 144 of file bufmgr.c.

Referenced by AsyncReadBuffers(), mdreadv(), and read_rel_block_ll().