From e5c1076a9961aba8175e723bd4cdda4bd438c288 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres@anarazel.de>
Date: Wed, 26 Oct 2022 14:44:02 -0700
Subject: [PATCH v1 08/12] bufmgr: Move relation extension handling into
 [Bulk]ExtendRelationBuffered()

---
 src/include/storage/buf_internals.h   |   5 +
 src/include/storage/bufmgr.h          |  13 +
 src/backend/storage/buffer/bufmgr.c   | 546 ++++++++++++++++++--------
 src/backend/storage/buffer/localbuf.c | 134 ++++++-
 src/backend/utils/probes.d            |   6 +-
 doc/src/sgml/monitoring.sgml          |  11 +-
 6 files changed, 546 insertions(+), 169 deletions(-)

diff --git a/src/include/storage/buf_internals.h b/src/include/storage/buf_internals.h
index 1cd6a968284..34b28eea906 100644
--- a/src/include/storage/buf_internals.h
+++ b/src/include/storage/buf_internals.h
@@ -420,6 +420,11 @@ extern PrefetchBufferResult PrefetchLocalBuffer(SMgrRelation smgr,
 												BlockNumber blockNum);
 extern BufferDesc *LocalBufferAlloc(SMgrRelation smgr, ForkNumber forkNum,
 									BlockNumber blockNum, bool *foundPtr);
+extern BlockNumber BulkExtendLocalRelationBuffered(SMgrRelation smgr,
+												   ForkNumber fork,
+												   ReadBufferMode mode,
+												   uint32 *num_pages,
+												   Buffer *buffers);
 extern void MarkLocalBufferDirty(Buffer buffer);
 extern void DropRelationLocalBuffers(RelFileLocator rlocator,
 									 ForkNumber forkNum,
diff --git a/src/include/storage/bufmgr.h b/src/include/storage/bufmgr.h
index 723243aeb96..a9cb876c777 100644
--- a/src/include/storage/bufmgr.h
+++ b/src/include/storage/bufmgr.h
@@ -123,6 +123,19 @@ extern void IncrBufferRefCount(Buffer buffer);
 extern void BufferCheckOneLocalPin(Buffer buffer);
 extern Buffer ReleaseAndReadBuffer(Buffer buffer, Relation relation,
 								   BlockNumber blockNum);
+extern Buffer ExtendRelationBuffered(Relation reln, struct SMgrRelationData *smgr,
+									 bool skip_extension_lock,
+									 char relpersistence,
+									 ForkNumber forkNum, ReadBufferMode mode,
+									 BufferAccessStrategy strategy);
+extern BlockNumber BulkExtendRelationBuffered(Relation rel, struct SMgrRelationData *smgr,
+											  bool skip_extension_lock,
+											  char relpersistence,
+											  ForkNumber fork, ReadBufferMode mode,
+											  BufferAccessStrategy strategy,
+											  uint32 *num_pages,
+											  uint32 num_locked_pages,
+											  Buffer *buffers);
 
 extern void InitBufferPoolAccess(void);
 extern void AtEOXact_Buffers(bool isCommit);
diff --git a/src/backend/storage/buffer/bufmgr.c b/src/backend/storage/buffer/bufmgr.c
index b1404bd77dd..196b7d1ff25 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -48,6 +48,7 @@
 #include "storage/buf_internals.h"
 #include "storage/bufmgr.h"
 #include "storage/ipc.h"
+#include "storage/lmgr.h"
 #include "storage/proc.h"
 #include "storage/smgr.h"
 #include "storage/standby.h"
@@ -459,6 +460,15 @@ static Buffer ReadBuffer_common(SMgrRelation smgr, char relpersistence,
 								ForkNumber forkNum, BlockNumber blockNum,
 								ReadBufferMode mode, BufferAccessStrategy strategy,
 								bool *hit);
+static BlockNumber BulkExtendSharedRelationBuffered(Relation rel,
+													SMgrRelation smgr,
+													bool skip_extension_lock,
+													char relpersistence,
+													ForkNumber fork, ReadBufferMode mode,
+													BufferAccessStrategy strategy,
+													uint32 *num_pages,
+													uint32 num_locked_pages,
+													Buffer *buffers);
 static bool PinBuffer(BufferDesc *buf, BufferAccessStrategy strategy);
 static void PinBuffer_Locked(BufferDesc *buf);
 static void UnpinBuffer(BufferDesc *buf);
@@ -793,6 +803,73 @@ ReadBufferWithoutRelcache(RelFileLocator rlocator, ForkNumber forkNum,
 							 mode, strategy, &hit);
 }
 
+/*
+ * Convenience wrapper around BulkExtendRelationBuffered() extending by one
+ * block.
+ */
+Buffer
+ExtendRelationBuffered(Relation rel, struct SMgrRelationData *smgr,
+					   bool skip_extension_lock,
+					   char relpersistence, ForkNumber forkNum,
+					   ReadBufferMode mode, BufferAccessStrategy strategy)
+{
+	Buffer buf;
+	uint32 num_pages = 1;
+
+	BulkExtendRelationBuffered(rel, smgr, skip_extension_lock, relpersistence,
+							   forkNum, mode, strategy, &num_pages, num_pages, &buf);
+
+	return buf;
+}
+
+
+BlockNumber
+BulkExtendRelationBuffered(Relation rel,
+						   SMgrRelation smgr,
+						   bool skip_extension_lock,
+						   char relpersistence,
+						   ForkNumber fork, ReadBufferMode mode,
+						   BufferAccessStrategy strategy,
+						   uint32 *num_pages,
+						   uint32 num_locked_pages,
+						   Buffer *buffers)
+{
+	BlockNumber first_block;
+
+	Assert(rel != NULL || smgr != NULL);
+	Assert(rel != NULL || skip_extension_lock);
+
+	if (smgr == NULL)
+		smgr = RelationGetSmgr(rel);
+
+	TRACE_POSTGRESQL_BUFFER_EXTEND_START(fork,
+										 smgr->smgr_rlocator.locator.spcOid,
+										 smgr->smgr_rlocator.locator.dbOid,
+										 smgr->smgr_rlocator.locator.relNumber,
+										 smgr->smgr_rlocator.backend,
+										 num_pages);
+
+	if (SmgrIsTemp(smgr))
+		first_block = BulkExtendLocalRelationBuffered(smgr,
+													  fork, mode,
+													  num_pages, buffers);
+	else
+		first_block = BulkExtendSharedRelationBuffered(rel, smgr,
+													   skip_extension_lock, relpersistence,
+													   fork, mode, strategy,
+													   num_pages, num_locked_pages,
+													   buffers);
+
+	TRACE_POSTGRESQL_BUFFER_EXTEND_DONE(fork,
+										smgr->smgr_rlocator.locator.spcOid,
+										smgr->smgr_rlocator.locator.dbOid,
+										smgr->smgr_rlocator.locator.relNumber,
+										smgr->smgr_rlocator.backend,
+										num_pages,
+										first_block);
+
+	return first_block;
+}
 
 /*
  * ReadBuffer_common -- common logic for all ReadBuffer variants
@@ -807,43 +884,32 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	BufferDesc *bufHdr;
 	Block		bufBlock;
 	bool		found;
-	bool		isExtend;
 	bool		isLocalBuf = SmgrIsTemp(smgr);
 
 	*hit = false;
 
+	/*
+	 * Backward compatibility path, most code should use
+	 * ExtendRelationBuffered() instead, as acquiring the extension lock
+	 * inside ExtendRelationBuffered() scales a lot better.
+	 */
+	if (unlikely(blockNum == P_NEW))
+		return ExtendRelationBuffered(NULL, smgr, true, relpersistence, forkNum, mode, strategy);
+
 	/* Make sure we will have room to remember the buffer pin */
 	ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
 
-	isExtend = (blockNum == P_NEW);
-
 	TRACE_POSTGRESQL_BUFFER_READ_START(forkNum, blockNum,
 									   smgr->smgr_rlocator.locator.spcOid,
 									   smgr->smgr_rlocator.locator.dbOid,
 									   smgr->smgr_rlocator.locator.relNumber,
-									   smgr->smgr_rlocator.backend,
-									   isExtend);
-
-	/* Substitute proper block number if caller asked for P_NEW */
-	if (isExtend)
-	{
-		blockNum = smgrnblocks(smgr, forkNum);
-		/* Fail if relation is already at maximum possible length */
-		if (blockNum == P_NEW)
-			ereport(ERROR,
-					(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
-					 errmsg("cannot extend relation %s beyond %u blocks",
-							relpath(smgr->smgr_rlocator, forkNum),
-							P_NEW)));
-	}
+									   smgr->smgr_rlocator.backend);
 
 	if (isLocalBuf)
 	{
 		bufHdr = LocalBufferAlloc(smgr, forkNum, blockNum, &found);
 		if (found)
 			pgBufferUsage.local_blks_hit++;
-		else if (isExtend)
-			pgBufferUsage.local_blks_written++;
 		else if (mode == RBM_NORMAL || mode == RBM_NORMAL_NO_LOG ||
 				 mode == RBM_ZERO_ON_ERROR)
 			pgBufferUsage.local_blks_read++;
@@ -858,8 +924,6 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 							 strategy, &found);
 		if (found)
 			pgBufferUsage.shared_blks_hit++;
-		else if (isExtend)
-			pgBufferUsage.shared_blks_written++;
 		else if (mode == RBM_NORMAL || mode == RBM_NORMAL_NO_LOG ||
 				 mode == RBM_ZERO_ON_ERROR)
 			pgBufferUsage.shared_blks_read++;
@@ -870,168 +934,88 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 	/* if it was already in the buffer pool, we're done */
 	if (found)
 	{
-		if (!isExtend)
-		{
-			/* Just need to update stats before we exit */
-			*hit = true;
-			VacuumPageHit++;
+		/* Just need to update stats before we exit */
+		*hit = true;
+		VacuumPageHit++;
 
-			if (VacuumCostActive)
-				VacuumCostBalance += VacuumCostPageHit;
+		if (VacuumCostActive)
+			VacuumCostBalance += VacuumCostPageHit;
 
-			TRACE_POSTGRESQL_BUFFER_READ_DONE(forkNum, blockNum,
-											  smgr->smgr_rlocator.locator.spcOid,
-											  smgr->smgr_rlocator.locator.dbOid,
-											  smgr->smgr_rlocator.locator.relNumber,
-											  smgr->smgr_rlocator.backend,
-											  isExtend,
-											  found);
-
-			/*
-			 * In RBM_ZERO_AND_LOCK mode the caller expects the page to be
-			 * locked on return.
-			 */
-			if (!isLocalBuf)
-			{
-				if (mode == RBM_ZERO_AND_LOCK)
-					LWLockAcquire(BufferDescriptorGetContentLock(bufHdr),
-								  LW_EXCLUSIVE);
-				else if (mode == RBM_ZERO_AND_CLEANUP_LOCK)
-					LockBufferForCleanup(BufferDescriptorGetBuffer(bufHdr));
-			}
-
-			return BufferDescriptorGetBuffer(bufHdr);
-		}
+		TRACE_POSTGRESQL_BUFFER_READ_DONE(forkNum, blockNum,
+										  smgr->smgr_rlocator.locator.spcOid,
+										  smgr->smgr_rlocator.locator.dbOid,
+										  smgr->smgr_rlocator.locator.relNumber,
+										  smgr->smgr_rlocator.backend,
+										  found);
 
 		/*
-		 * We get here only in the corner case where we are trying to extend
-		 * the relation but we found a pre-existing buffer marked BM_VALID.
-		 * This can happen because mdread doesn't complain about reads beyond
-		 * EOF (when zero_damaged_pages is ON) and so a previous attempt to
-		 * read a block beyond EOF could have left a "valid" zero-filled
-		 * buffer.  Unfortunately, we have also seen this case occurring
-		 * because of buggy Linux kernels that sometimes return an
-		 * lseek(SEEK_END) result that doesn't account for a recent write. In
-		 * that situation, the pre-existing buffer would contain valid data
-		 * that we don't want to overwrite.  Since the legitimate case should
-		 * always have left a zero-filled buffer, complain if not PageIsNew.
+		 * In RBM_ZERO_AND_LOCK mode the caller expects the page to be
+		 * locked on return.
 		 */
-		bufBlock = isLocalBuf ? LocalBufHdrGetBlock(bufHdr) : BufHdrGetBlock(bufHdr);
-		if (!PageIsNew((Page) bufBlock))
-			ereport(ERROR,
-					(errmsg("unexpected data beyond EOF in block %u of relation %s",
-							blockNum, relpath(smgr->smgr_rlocator, forkNum)),
-					 errhint("This has been seen to occur with buggy kernels; consider updating your system.")));
-
-		/*
-		 * We *must* do smgrextend before succeeding, else the page will not
-		 * be reserved by the kernel, and the next P_NEW call will decide to
-		 * return the same page.  Clear the BM_VALID bit, do the StartBufferIO
-		 * call that BufferAlloc didn't, and proceed.
-		 */
-		if (isLocalBuf)
+		if (!isLocalBuf)
 		{
-			/* Only need to adjust flags */
-			uint32		buf_state = pg_atomic_read_u32(&bufHdr->state);
-
-			Assert(buf_state & BM_VALID);
-			buf_state &= ~BM_VALID;
-			pg_atomic_unlocked_write_u32(&bufHdr->state, buf_state);
+			if (mode == RBM_ZERO_AND_LOCK)
+				LWLockAcquire(BufferDescriptorGetContentLock(bufHdr),
+							  LW_EXCLUSIVE);
+			else if (mode == RBM_ZERO_AND_CLEANUP_LOCK)
+				LockBufferForCleanup(BufferDescriptorGetBuffer(bufHdr));
 		}
-		else
-		{
-			/*
-			 * Loop to handle the very small possibility that someone re-sets
-			 * BM_VALID between our clearing it and StartBufferIO inspecting
-			 * it.
-			 */
-			do
-			{
-				uint32		buf_state = LockBufHdr(bufHdr);
 
-				Assert(buf_state & BM_VALID);
-				buf_state &= ~BM_VALID;
-				UnlockBufHdr(bufHdr, buf_state);
-			} while (!StartBufferIO(bufHdr, true));
-		}
+		return BufferDescriptorGetBuffer(bufHdr);
 	}
 
 	/*
 	 * if we have gotten to this point, we have allocated a buffer for the
 	 * page but its contents are not yet valid.  IO_IN_PROGRESS is set for it,
 	 * if it's a shared buffer.
-	 *
-	 * Note: if smgrextend fails, we will end up with a buffer that is
-	 * allocated but not marked BM_VALID.  P_NEW will still select the same
-	 * block number (because the relation didn't get any longer on disk) and
-	 * so future attempts to extend the relation will find the same buffer (if
-	 * it's not been recycled) but come right back here to try smgrextend
-	 * again.
 	 */
 	Assert(!(pg_atomic_read_u32(&bufHdr->state) & BM_VALID));	/* spinlock not needed */
 
 	bufBlock = isLocalBuf ? LocalBufHdrGetBlock(bufHdr) : BufHdrGetBlock(bufHdr);
 
-	if (isExtend)
-	{
-		/* new buffers are zero-filled */
+	/*
+	 * Read in the page, unless the caller intends to overwrite it and
+	 * just wants us to allocate a buffer.
+	 */
+	if (mode == RBM_ZERO_AND_LOCK || mode == RBM_ZERO_AND_CLEANUP_LOCK)
 		MemSet((char *) bufBlock, 0, BLCKSZ);
-		/* don't set checksum for all-zero page */
-		smgrextend(smgr, forkNum, blockNum, (char *) bufBlock, false);
-
-		/*
-		 * NB: we're *not* doing a ScheduleBufferTagForWriteback here;
-		 * although we're essentially performing a write. At least on linux
-		 * doing so defeats the 'delayed allocation' mechanism, leading to
-		 * increased file fragmentation.
-		 */
-	}
 	else
 	{
-		/*
-		 * Read in the page, unless the caller intends to overwrite it and
-		 * just wants us to allocate a buffer.
-		 */
-		if (mode == RBM_ZERO_AND_LOCK || mode == RBM_ZERO_AND_CLEANUP_LOCK)
-			MemSet((char *) bufBlock, 0, BLCKSZ);
-		else
+		instr_time	io_start,
+					io_time;
+
+		if (track_io_timing)
+			INSTR_TIME_SET_CURRENT(io_start);
+
+		smgrread(smgr, forkNum, blockNum, (char *) bufBlock);
+
+		if (track_io_timing)
 		{
-			instr_time	io_start,
-						io_time;
+			INSTR_TIME_SET_CURRENT(io_time);
+			INSTR_TIME_SUBTRACT(io_time, io_start);
+			pgstat_count_buffer_read_time(INSTR_TIME_GET_MICROSEC(io_time));
+			INSTR_TIME_ADD(pgBufferUsage.blk_read_time, io_time);
+		}
 
-			if (track_io_timing)
-				INSTR_TIME_SET_CURRENT(io_start);
-
-			smgrread(smgr, forkNum, blockNum, (char *) bufBlock);
-
-			if (track_io_timing)
+		/* check for garbage data */
+		if (!PageIsVerifiedExtended((Page) bufBlock, blockNum,
+									PIV_LOG_WARNING | PIV_REPORT_STAT))
+		{
+			if (mode == RBM_ZERO_ON_ERROR || zero_damaged_pages)
 			{
-				INSTR_TIME_SET_CURRENT(io_time);
-				INSTR_TIME_SUBTRACT(io_time, io_start);
-				pgstat_count_buffer_read_time(INSTR_TIME_GET_MICROSEC(io_time));
-				INSTR_TIME_ADD(pgBufferUsage.blk_read_time, io_time);
-			}
-
-			/* check for garbage data */
-			if (!PageIsVerifiedExtended((Page) bufBlock, blockNum,
-										PIV_LOG_WARNING | PIV_REPORT_STAT))
-			{
-				if (mode == RBM_ZERO_ON_ERROR || zero_damaged_pages)
-				{
-					ereport(WARNING,
-							(errcode(ERRCODE_DATA_CORRUPTED),
-							 errmsg("invalid page in block %u of relation %s; zeroing out page",
-									blockNum,
-									relpath(smgr->smgr_rlocator, forkNum))));
-					MemSet((char *) bufBlock, 0, BLCKSZ);
-				}
-				else
-					ereport(ERROR,
-							(errcode(ERRCODE_DATA_CORRUPTED),
-							 errmsg("invalid page in block %u of relation %s",
-									blockNum,
-									relpath(smgr->smgr_rlocator, forkNum))));
+				ereport(WARNING,
+						(errcode(ERRCODE_DATA_CORRUPTED),
+						 errmsg("invalid page in block %u of relation %s; zeroing out page",
+								blockNum,
+								relpath(smgr->smgr_rlocator, forkNum))));
+				MemSet((char *) bufBlock, 0, BLCKSZ);
 			}
+			else
+				ereport(ERROR,
+						(errcode(ERRCODE_DATA_CORRUPTED),
+						 errmsg("invalid page in block %u of relation %s",
+								blockNum,
+								relpath(smgr->smgr_rlocator, forkNum))));
 		}
 	}
 
@@ -1074,7 +1058,6 @@ ReadBuffer_common(SMgrRelation smgr, char relpersistence, ForkNumber forkNum,
 									  smgr->smgr_rlocator.locator.dbOid,
 									  smgr->smgr_rlocator.locator.relNumber,
 									  smgr->smgr_rlocator.backend,
-									  isExtend,
 									  found);
 
 	return BufferDescriptorGetBuffer(bufHdr);
@@ -1617,6 +1600,251 @@ again:
 
 	return cur_buf;
 }
+
+/*
+ * Limit the number of pins a batch operation may additionally acquire, to
+ * avoid running out of pinnable buffers.
+ *
+ * One additional pin is always allowed, as otherwise the operation likely
+ * cannot be performed at all.
+ *
+ * The number of allowed pins for a backend is computed based on
+ * shared_buffers and the maximum number of connections possible. That's very
+ * pessimistic, but oustide of toy-sized shared_buffers it should allow
+ * sufficient pins.
+ */
+static void
+LimitAdditionalPins(uint32 *additional_pins)
+{
+	uint32 max_backends;
+	int max_proportional_pins;
+
+	if (*additional_pins <= 1)
+		return;
+
+	max_backends = MaxBackends + NUM_AUXILIARY_PROCS;
+	max_proportional_pins = NBuffers / max_backends;
+
+	/*
+	 * Subtract the approximate number of buffers already pinned by this
+	 * backend. We get the number of "overflowed" pins for free, but don't
+	 * know the number of pins in PrivateRefCountArray. The cost of
+	 * calculating that exactly doesn't seem worth it, so just assume the max.
+	 */
+	max_proportional_pins -= PrivateRefCountOverflowed + REFCOUNT_ARRAY_ENTRIES;
+
+	if (max_proportional_pins < 0)
+		max_proportional_pins = 1;
+
+	if (*additional_pins > max_proportional_pins)
+		*additional_pins = max_proportional_pins;
+}
+
+static BlockNumber
+BulkExtendSharedRelationBuffered(Relation rel,
+								 SMgrRelation smgr,
+								 bool skip_extension_lock,
+								 char relpersistence,
+								 ForkNumber fork, ReadBufferMode mode,
+								 BufferAccessStrategy strategy,
+								 uint32 *num_pages,
+								 uint32 num_locked_pages,
+								 Buffer *buffers)
+{
+	BlockNumber first_block;
+
+	LimitAdditionalPins(num_pages);
+
+	/*
+	 * FIXME: limit num_pages / buffers based on NBuffers / MaxBackends or
+	 * such. Also keep MAX_SIMUL_LWLOCKS in mind.
+	 */
+
+	pgBufferUsage.shared_blks_written += *num_pages;
+
+	/*
+	 * Acquire victim buffers for extension without holding extension
+	 * lock. Writing out victim buffers is the most expensive part of
+	 * extending the relation, particularly when doing so requires WAL
+	 * flushes. Zeroing out the buffers is also quite expensive, so do that
+	 * before holding the extension lock as well.
+	 *
+	 * These pages are pinned by us and not valid. While we hold the pin
+	 * they can't be acquired as victim buffers by another backend.
+	 */
+	for (uint32 i = 0; i < *num_pages; i++)
+	{
+		Block		buf_block;
+
+		buffers[i] = GetVictimBuffer(strategy);
+		buf_block = BufHdrGetBlock(GetBufferDescriptor(buffers[i] - 1));
+
+		/* new buffers are zero-filled */
+		MemSet((char *) buf_block, 0, BLCKSZ);
+	}
+
+	if (!skip_extension_lock)
+		LockRelationForExtension(rel, ExclusiveLock);
+
+	first_block = smgrnblocks(smgr, fork);
+
+	/* Fail if relation is already at maximum possible length */
+	if ((uint64) first_block + *num_pages >= MaxBlockNumber)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+				 errmsg("cannot extend relation %s beyond %u blocks",
+						relpath(smgr->smgr_rlocator, fork),
+						MaxBlockNumber)));
+
+	/*
+	 * Insert buffers into buffer table, mark as IO_IN_PROGRESS.
+	 *
+	 * This needs to happen before we extend the relation, because as soon as
+	 * we do, other backends can start to read in those pages.
+	 */
+	for (int i = 0; i < *num_pages; i++)
+	{
+		Buffer		victim_buf = buffers[i];
+		BufferDesc *victim_buf_hdr = GetBufferDescriptor(victim_buf - 1);
+		BufferTag	tag;
+		uint32		hash;
+		LWLock     *partition_lock;
+		int			existing_id;
+
+		InitBufferTag(&tag, &smgr->smgr_rlocator.locator, fork, first_block + i);
+		hash = BufTableHashCode(&tag);
+		partition_lock = BufMappingPartitionLock(hash);
+
+		LWLockAcquire(partition_lock, LW_EXCLUSIVE);
+
+		existing_id = BufTableInsert(&tag, hash, victim_buf_hdr->buf_id);
+
+		/*
+		 * We get here only in the corner case where we are trying to extend
+		 * the relation but we found a pre-existing buffer. This can happen
+		 * because a prior attempt at extending the relation failed, and
+		 * because mdread doesn't complain about reads beyond EOF (when
+		 * zero_damaged_pages is ON) and so a previous attempt to read a block
+		 * beyond EOF could have left a "valid" zero-filled buffer.
+		 * Unfortunately, we have also seen this case occurring because of
+		 * buggy Linux kernels that sometimes return an lseek(SEEK_END) result
+		 * that doesn't account for a recent write. In that situation, the
+		 * pre-existing buffer would contain valid data that we don't want to
+		 * overwrite.  Since the legitimate cases should always have left a
+		 * zero-filled buffer, complain if not PageIsNew.
+		 */
+		if (existing_id >= 0)
+		{
+			BufferDesc *existing_hdr = GetBufferDescriptor(existing_id);
+			Block		buf_block;
+			bool		valid;
+
+			/*
+			 * Pin the existing buffer before releasing the partition lock,
+			 * preventing it from being evicted.
+			 */
+			valid = PinBuffer(existing_hdr, strategy);
+
+			LWLockRelease(partition_lock);
+
+			/*
+			 * The victim buffer we acquired peviously is clean and unused,
+			 * let it be found again quickly
+			 */
+			StrategyFreeBuffer(victim_buf_hdr);
+			UnpinBuffer(victim_buf_hdr);
+
+			buffers[i] = BufferDescriptorGetBuffer(existing_hdr);
+			buf_block = BufHdrGetBlock(existing_hdr);
+
+			if (valid && !PageIsNew((Page) buf_block))
+				ereport(ERROR,
+						(errmsg("unexpected data beyond EOF in block %u of relation %s",
+								existing_hdr->tag.blockNum, relpath(smgr->smgr_rlocator, fork)),
+						 errhint("This has been seen to occur with buggy kernels; consider updating your system.")));
+
+			/*
+			 * We *must* do smgr[zero]extend before succeeding, else the page
+			 * will not be reserved by the kernel, and the next P_NEW call
+			 * will decide to return the same page.  Clear the BM_VALID bit,
+			 * do StartBufferIO() and proceed.
+			 *
+			 * Loop to handle the very small possibility that someone re-sets
+			 * BM_VALID between our clearing it and StartBufferIO inspecting
+			 * it.
+			 */
+			do
+			{
+				uint32		buf_state = LockBufHdr(existing_hdr);
+
+				buf_state &= ~BM_VALID;
+				UnlockBufHdr(existing_hdr, buf_state);
+			} while (!StartBufferIO(existing_hdr, true));
+		}
+		else
+		{
+			uint32		buf_state;
+
+			buf_state = LockBufHdr(victim_buf_hdr);
+
+			/* some sanity checks while we hold the buffer header lock */
+			Assert(!(buf_state & (BM_VALID | BM_TAG_VALID | BM_DIRTY | BM_JUST_DIRTIED)));
+			Assert(BUF_STATE_GET_REFCOUNT(buf_state) == 1);
+
+			victim_buf_hdr->tag = tag;
+
+			buf_state |= BM_TAG_VALID | BUF_USAGECOUNT_ONE;
+			if (relpersistence == RELPERSISTENCE_PERMANENT || fork == INIT_FORKNUM)
+				buf_state |= BM_PERMANENT;
+
+			UnlockBufHdr(victim_buf_hdr, buf_state);
+
+			LWLockRelease(partition_lock);
+
+			/* XXX: could combine the locked operations in it with the above */
+			StartBufferIO(victim_buf_hdr, true);
+		}
+	}
+
+	/*
+	 * Note: if smgzerorextend fails, we will end up with buffers that are
+	 * allocated but not marked BM_VALID.  The next relation extension will
+	 * still select the same block number (because the relation didn't get any
+	 * longer on disk) and so future attempts to extend the relation will find
+	 * the same buffers (if they have not been recycled) but come right back
+	 * here to try smgrzeroextend again.
+	 *
+	 * We don't need to set checksum for all-zero pages.
+	 */
+	smgrzeroextend(smgr, fork, first_block, *num_pages, false);
+
+	/*
+	 * Release the file-extension lock; it's now OK for someone else to extend
+	 * the relation some more.
+	 *
+	 * We remove IO_IN_PROGRESS after this, as zeroing the buffer contents and
+	 * waking up waiting backends waiting can take noticeable time.
+	 */
+	if (!skip_extension_lock)
+		UnlockRelationForExtension(rel, ExclusiveLock);
+
+	/* Set BM_VALID, terminate IO, and wake up any waiters */
+	for (int i = 0; i < *num_pages; i++)
+	{
+		Buffer		buf = buffers[i];
+		BufferDesc *buf_hdr = GetBufferDescriptor(buf - 1);
+
+		if (i < num_locked_pages &&
+			(mode == RBM_ZERO_AND_LOCK || mode == RBM_ZERO_AND_CLEANUP_LOCK))
+			LWLockAcquire(BufferDescriptorGetContentLock(buf_hdr), LW_EXCLUSIVE);
+
+		TerminateBufferIO(buf_hdr, false, BM_VALID);
+	}
+
+	return first_block;
+
+}
+
 /*
  * MarkBufferDirty
  *
diff --git a/src/backend/storage/buffer/localbuf.c b/src/backend/storage/buffer/localbuf.c
index 6a3f325ea6d..5e93d8ba839 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
@@ -48,6 +48,9 @@ static int	nextFreeLocalBufId = 0;
 
 static HTAB *LocalBufHash = NULL;
 
+/* number of local buffers pinned at least once */
+static int	NLocalPinnedBuffers = 0;
+
 
 static void InitLocalBuffers(void);
 static Block GetLocalBufferStorage(void);
@@ -270,6 +273,132 @@ GetLocalVictimBuffer(void)
 	return BufferDescriptorGetBuffer(bufHdr);
 }
 
+/* see LimitAdditionalPins() */
+static void
+LimitAdditionalLocalPins(uint32 *additional_pins)
+{
+	uint32 max_pins;
+
+	if (*additional_pins <= 1)
+		return;
+
+	/*
+	 * In contrast to LimitAdditionalPins() other backends don't play a role
+	 * here. We can allow up to NLocBuffer pins in total.
+	 */
+	max_pins = (NLocBuffer - NLocalPinnedBuffers);
+
+	if (*additional_pins >= max_pins)
+		*additional_pins = max_pins;
+}
+
+BlockNumber
+BulkExtendLocalRelationBuffered(SMgrRelation smgr,
+								ForkNumber fork,
+								ReadBufferMode mode,
+								uint32 *num_pages,
+								Buffer *buffers)
+{
+	BlockNumber first_block;
+
+	/* Initialize local buffers if first request in this session */
+	if (LocalBufHash == NULL)
+		InitLocalBuffers();
+
+	LimitAdditionalLocalPins(num_pages);
+
+	pgBufferUsage.temp_blks_written += *num_pages;
+
+	for (uint32 i = 0; i < *num_pages; i++)
+	{
+		BufferDesc *buf_hdr;
+		Block		buf_block;
+
+		buffers[i] = GetLocalVictimBuffer();
+		buf_hdr = GetLocalBufferDescriptor(-(buffers[i] + 1));
+		buf_block = LocalBufHdrGetBlock(buf_hdr);
+
+		/* new buffers are zero-filled */
+		MemSet((char *) buf_block, 0, BLCKSZ);
+	}
+
+	first_block = smgrnblocks(smgr, fork);
+
+	/* Fail if relation is already at maximum possible length */
+	if ((uint64) first_block + *num_pages >= MaxBlockNumber)
+		ereport(ERROR,
+				(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
+				 errmsg("cannot extend relation %s beyond %u blocks",
+						relpath(smgr->smgr_rlocator, fork),
+						MaxBlockNumber)));
+
+	for (int i = 0; i < *num_pages; i++)
+	{
+		int			victim_buf_id;
+		BufferDesc *victim_buf_hdr;
+		BufferTag	tag;
+		LocalBufferLookupEnt *hresult;
+		bool		found;
+
+		victim_buf_id = -(buffers[i] + 1);
+		victim_buf_hdr = GetLocalBufferDescriptor(victim_buf_id);
+
+		InitBufferTag(&tag, &smgr->smgr_rlocator.locator, fork, first_block + i);
+
+		hresult = (LocalBufferLookupEnt *)
+			hash_search(LocalBufHash, (void *) &tag, HASH_ENTER, &found);
+		if (found)
+		{
+			BufferDesc *existing_hdr = GetLocalBufferDescriptor(hresult->id);
+			uint32		buf_state;
+
+			UnpinLocalBuffer(BufferDescriptorGetBuffer(victim_buf_hdr));
+
+			existing_hdr = GetLocalBufferDescriptor(hresult->id);
+			PinLocalBuffer(existing_hdr, false);
+			buffers[i] = BufferDescriptorGetBuffer(existing_hdr);
+
+			buf_state = pg_atomic_read_u32(&existing_hdr->state);
+			Assert(buf_state & BM_TAG_VALID);
+			Assert(!(buf_state & BM_DIRTY));
+			buf_state &= BM_VALID;
+			pg_atomic_unlocked_write_u32(&existing_hdr->state, buf_state);
+		}
+		else
+		{
+			uint32		buf_state = pg_atomic_read_u32(&victim_buf_hdr->state);
+
+			Assert(!(buf_state & (BM_VALID | BM_TAG_VALID | BM_DIRTY | BM_JUST_DIRTIED)));
+
+			victim_buf_hdr->tag = tag;
+
+			buf_state |= BM_TAG_VALID | BUF_USAGECOUNT_ONE;
+
+			pg_atomic_unlocked_write_u32(&victim_buf_hdr->state, buf_state);
+
+			hresult->id = victim_buf_id;
+		}
+	}
+
+	/* actually extend relation */
+	smgrzeroextend(smgr, fork, first_block, *num_pages, false);
+
+	for (int i = 0; i < *num_pages; i++)
+	{
+		Buffer		buf = buffers[i];
+		BufferDesc *buf_hdr;
+		uint32		buf_state;
+
+		buf_hdr = GetLocalBufferDescriptor(-(buf + 1));
+
+		buf_state = pg_atomic_read_u32(&buf_hdr->state);
+		buf_state |= BM_VALID;
+		pg_atomic_unlocked_write_u32(&buf_hdr->state, buf_state);
+	}
+
+	return first_block;
+}
+
 /*
  * MarkLocalBufferDirty -
  *	  mark a local buffer dirty
@@ -486,6 +615,7 @@ PinLocalBuffer(BufferDesc *buf_hdr, bool adjust_usagecount)
 
 	if (LocalRefCount[bufid] == 0)
 	{
+		NLocalPinnedBuffers++;
 		if (adjust_usagecount &&
 			BUF_STATE_GET_USAGECOUNT(buf_state) < BM_MAX_USAGE_COUNT)
 		{
@@ -507,9 +637,11 @@ UnpinLocalBuffer(Buffer buffer)
 
 	Assert(BufferIsLocal(buffer));
 	Assert(LocalRefCount[buffid] > 0);
+	Assert(NLocalPinnedBuffers > 0);
 
 	ResourceOwnerForgetBuffer(CurrentResourceOwner, buffer);
-	LocalRefCount[buffid]--;
+	if (--LocalRefCount[buffid] == 0)
+		NLocalPinnedBuffers--;
 }
 
 /*
diff --git a/src/backend/utils/probes.d b/src/backend/utils/probes.d
index 3ebbcf88ebe..12b4d995cfa 100644
--- a/src/backend/utils/probes.d
+++ b/src/backend/utils/probes.d
@@ -55,10 +55,12 @@ provider postgresql {
 	probe sort__start(int, bool, int, int, bool, int);
 	probe sort__done(bool, long);
 
-	probe buffer__read__start(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool);
-	probe buffer__read__done(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool, bool);
+	probe buffer__read__start(ForkNumber, BlockNumber, Oid, Oid, Oid, int);
+	probe buffer__read__done(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool);
 	probe buffer__flush__start(ForkNumber, BlockNumber, Oid, Oid, Oid);
 	probe buffer__flush__done(ForkNumber, BlockNumber, Oid, Oid, Oid);
+	probe buffer__extend__start(ForkNumber, Oid, Oid, Oid, int, int);
+	probe buffer__extend__done(ForkNumber, Oid, Oid, Oid, int, int, BlockNumber);
 
 	probe buffer__checkpoint__start(int);
 	probe buffer__checkpoint__sync__start();
diff --git a/doc/src/sgml/monitoring.sgml b/doc/src/sgml/monitoring.sgml
index e5d622d5147..b58a30ef5a8 100644
--- a/doc/src/sgml/monitoring.sgml
+++ b/doc/src/sgml/monitoring.sgml
@@ -7344,7 +7344,7 @@ FROM pg_stat_get_backend_idset() AS backendid;
     </row>
     <row>
      <entry><literal>buffer-read-start</literal></entry>
-     <entry><literal>(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool)</literal></entry>
+     <entry><literal>(ForkNumber, BlockNumber, Oid, Oid, Oid, int)</literal></entry>
      <entry>Probe that fires when a buffer read is started.
       arg0 and arg1 contain the fork and block numbers of the page (but
       arg1 will be -1 if this is a relation extension request).
@@ -7352,12 +7352,11 @@ FROM pg_stat_get_backend_idset() AS backendid;
       identifying the relation.
       arg5 is the ID of the backend which created the temporary relation for a
       local buffer, or <symbol>InvalidBackendId</symbol> (-1) for a shared buffer.
-      arg6 is true for a relation extension request, false for normal
-      read.</entry>
+      </entry>
     </row>
     <row>
      <entry><literal>buffer-read-done</literal></entry>
-     <entry><literal>(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool, bool)</literal></entry>
+     <entry><literal>(ForkNumber, BlockNumber, Oid, Oid, Oid, int, bool)</literal></entry>
      <entry>Probe that fires when a buffer read is complete.
       arg0 and arg1 contain the fork and block numbers of the page (if this
       is a relation extension request, arg1 now contains the block number
@@ -7366,9 +7365,7 @@ FROM pg_stat_get_backend_idset() AS backendid;
       identifying the relation.
       arg5 is the ID of the backend which created the temporary relation for a
       local buffer, or <symbol>InvalidBackendId</symbol> (-1) for a shared buffer.
-      arg6 is true for a relation extension request, false for normal
-      read.
-      arg7 is true if the buffer was found in the pool, false if not.</entry>
+      arg6 is true if the buffer was found in the pool, false if not.</entry>
     </row>
     <row>
      <entry><literal>buffer-flush-start</literal></entry>
-- 
2.38.0

