Re: refactoring relation extension and BufferAlloc(), faster COPY

From: Melanie Plageman <melanieplageman(at)gmail(dot)com>
To: Andres Freund <andres(at)anarazel(dot)de>
Cc: Heikki Linnakangas <hlinnaka(at)iki(dot)fi>, Alvaro Herrera <alvherre(at)alvh(dot)no-ip(dot)org>, vignesh C <vignesh21(at)gmail(dot)com>, pgsql-hackers(at)postgresql(dot)org, Thomas Munro <thomas(dot)munro(at)gmail(dot)com>, Yura Sokolov <y(dot)sokolov(at)postgrespro(dot)ru>, Robert Haas <robertmhaas(at)gmail(dot)com>
Subject: Re: refactoring relation extension and BufferAlloc(), faster COPY
Date: 2023-03-26 21:42:45
Message-ID: CAAKRu_aW2SX_LWtwHgfnqYpBrunMLfE9PD6-ioPpkh92XH0qpg@mail.gmail.com
Views: Raw Message | Whole Thread | Download mbox | Resend email
Thread:
Lists: pgsql-hackers

Hi,

Below is my review of a slightly older version than you just posted --
much of it you may have already addressed.

From 3a6c3f41000e057bae12ab4431e6bb1c5f3ec4b0 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Mon, 20 Mar 2023 21:57:40 -0700
Subject: [PATCH v5 01/15] createdb-using-wal-fixes

This could use a more detailed commit message -- I don't really get what
it is doing

From 6faba69c241fd5513022bb042c33af09d91e84a6 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Wed, 1 Jul 2020 19:06:45 -0700
Subject: [PATCH v5 02/15] Add some error checking around pinning

---
src/backend/storage/buffer/bufmgr.c | 40 ++++++++++++++++++++---------
src/include/storage/bufmgr.h | 1 +
2 files changed, 29 insertions(+), 12 deletions(-)

diff --git a/src/backend/storage/buffer/bufmgr.c
b/src/backend/storage/buffer/bufmgr.c
index 95212a3941..fa20fab5a2 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -4283,6 +4287,25 @@ ConditionalLockBuffer(Buffer buffer)
LW_EXCLUSIVE);
}

+void
+BufferCheckOneLocalPin(Buffer buffer)
+{
+ if (BufferIsLocal(buffer))
+ {
+ /* There should be exactly one pin */
+ if (LocalRefCount[-buffer - 1] != 1)
+ elog(ERROR, "incorrect local pin count: %d",
+ LocalRefCount[-buffer - 1]);
+ }
+ else
+ {
+ /* There should be exactly one local pin */
+ if (GetPrivateRefCount(buffer) != 1)

I'd rather this be an else if (was already like this, but, no reason not
to change it now)

+ elog(ERROR, "incorrect local pin count: %d",
+ GetPrivateRefCount(buffer));
+ }
+}

From 00d3044770478eea31e00fee8d1680f22ca6adde Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Mon, 27 Feb 2023 17:36:37 -0800
Subject: [PATCH v5 04/15] Add smgrzeroextend(), FileZero(), FileFallocate()

diff --git a/src/backend/storage/file/fd.c b/src/backend/storage/file/fd.c
index 9fd8444ed4..c34ed41d52 100644
--- a/src/backend/storage/file/fd.c
+++ b/src/backend/storage/file/fd.c
@@ -2206,6 +2206,92 @@ FileSync(File file, uint32 wait_event_info)
return returnCode;
}

+/*
+ * Zero a region of the file.
+ *
+ * Returns 0 on success, -1 otherwise. In the latter case errno is set to the
+ * appropriate error.
+ */
+int
+FileZero(File file, off_t offset, off_t amount, uint32 wait_event_info)
+{
+ int returnCode;
+ ssize_t written;
+
+ Assert(FileIsValid(file));
+ returnCode = FileAccess(file);
+ if (returnCode < 0)
+ return returnCode;
+
+ pgstat_report_wait_start(wait_event_info);
+ written = pg_pwrite_zeros(VfdCache[file].fd, amount, offset);
+ pgstat_report_wait_end();
+
+ if (written < 0)
+ return -1;
+ else if (written != amount)

this doesn't need to be an else if

+ {
+ /* if errno is unset, assume problem is no disk space */
+ if (errno == 0)
+ errno = ENOSPC;
+ return -1;
+ }

+int
+FileFallocate(File file, off_t offset, off_t amount, uint32 wait_event_info)
+{
+#ifdef HAVE_POSIX_FALLOCATE
+ int returnCode;
+
+ Assert(FileIsValid(file));
+ returnCode = FileAccess(file);
+ if (returnCode < 0)
+ return returnCode;
+
+ pgstat_report_wait_start(wait_event_info);
+ returnCode = posix_fallocate(VfdCache[file].fd, offset, amount);
+ pgstat_report_wait_end();
+
+ if (returnCode == 0)
+ return 0;
+
+ /* for compatibility with %m printing etc */
+ errno = returnCode;
+
+ /*
+ * Return in cases of a "real" failure, if fallocate is not supported,
+ * fall through to the FileZero() backed implementation.
+ */
+ if (returnCode != EINVAL && returnCode != EOPNOTSUPP)
+ return returnCode;

I'm pretty sure you can just delete the below if statement

+ if (returnCode == 0 ||
+ (returnCode != EINVAL && returnCode != EINVAL))
+ return returnCode;

diff --git a/src/backend/storage/smgr/md.c b/src/backend/storage/smgr/md.c
index 352958e1fe..59a65a8305 100644
--- a/src/backend/storage/smgr/md.c
+++ b/src/backend/storage/smgr/md.c
@@ -28,6 +28,7 @@
#include "access/xlog.h"
#include "access/xlogutils.h"
#include "commands/tablespace.h"
+#include "common/file_utils.h"
#include "miscadmin.h"
#include "pg_trace.h"
#include "pgstat.h"
@@ -500,6 +501,116 @@ mdextend(SMgrRelation reln, ForkNumber forknum,
BlockNumber blocknum,
Assert(_mdnblocks(reln, forknum, v) <= ((BlockNumber) RELSEG_SIZE));
}

+/*
+ * mdzeroextend() -- Add ew zeroed out blocks to the specified relation.

not sure what ew is

+ *
+ * Similar to mdrextend(), except the relation can be extended by

mdrextend->mdextend

+ * multiple blocks at once, and that the added blocks will be
filled with

I would lose the comma and just say "and the added blocks will be filled..."

+void
+mdzeroextend(SMgrRelation reln, ForkNumber forknum,
+ BlockNumber blocknum, int nblocks, bool skipFsync)

So, I think there are a few too many local variables in here, and it
actually makes it more confusing.
Assuming you would like to keep the input parameters blocknum and
nblocks unmodified for debugging/other reasons, here is a suggested
refactor of this function
Also, I think you can combine the two error cases (I don't know if the
user cares what you were trying to extend the file with). I've done this
below also.

void
mdzeroextend(SMgrRelation reln, ForkNumber forknum,
BlockNumber blocknum, int nblocks, bool skipFsync)
{
MdfdVec *v;
BlockNumber curblocknum = blocknum;
int remblocks = nblocks;

Assert(nblocks > 0);

/* This assert is too expensive to have on normally ... */
#ifdef CHECK_WRITE_VS_EXTEND
Assert(blocknum >= mdnblocks(reln, forknum));
#endif

/*
* If a relation manages to grow to 2^32-1 blocks, refuse to extend it any
* more --- we mustn't create a block whose number actually is
* InvalidBlockNumber or larger.
*/
if ((uint64) blocknum + nblocks >= (uint64) InvalidBlockNumber)
ereport(ERROR,
(errcode(ERRCODE_PROGRAM_LIMIT_EXCEEDED),
errmsg("cannot extend file \"%s\" beyond %u blocks",
relpath(reln->smgr_rlocator, forknum),
InvalidBlockNumber)));

while (remblocks > 0)
{
int segstartblock = curblocknum % ((BlockNumber)
RELSEG_SIZE);
int numblocks = remblocks;
off_t seekpos = (off_t) BLCKSZ * segstartblock;
int ret;

if (segstartblock + remblocks > RELSEG_SIZE)
numblocks = RELSEG_SIZE - segstartblock;

v = _mdfd_getseg(reln, forknum, curblocknum, skipFsync,
EXTENSION_CREATE);

/*
* If available and useful, use posix_fallocate() (via FileAllocate())
* to extend the relation. That's often more efficient than using
* write(), as it commonly won't cause the kernel to allocate page
* cache space for the extended pages.
*
* However, we don't use FileAllocate() for small extensions, as it
* defeats delayed allocation on some filesystems. Not clear where
* that decision should be made though? For now just use a cutoff of
* 8, anything between 4 and 8 worked OK in some local testing.
*/
if (numblocks > 8)
ret = FileFallocate(v->mdfd_vfd,
seekpos, (off_t) BLCKSZ * numblocks,
WAIT_EVENT_DATA_FILE_EXTEND);
else
/*
* Even if we don't want to use fallocate, we can still extend a
* bit more efficiently than writing each 8kB block individually.
* pg_pwrite_zeroes() (via FileZero()) uses
* pg_pwritev_with_retry() to avoid multiple writes or needing a
* zeroed buffer for the whole length of the extension.
*/
ret = FileZero(v->mdfd_vfd,
seekpos, (off_t) BLCKSZ * numblocks,
WAIT_EVENT_DATA_FILE_EXTEND);

if (ret != 0)
ereport(ERROR,
errcode_for_file_access(),
errmsg("could not extend file \"%s\": %m",
FilePathName(v->mdfd_vfd)),
errhint("Check free disk space."));

if (!skipFsync && !SmgrIsTemp(reln))
register_dirty_segment(reln, forknum, v);

Assert(_mdnblocks(reln, forknum, v) <= ((BlockNumber) RELSEG_SIZE));

remblocks -= numblocks;
curblocknum += numblocks;
}
}

diff --git a/src/backend/storage/smgr/smgr.c b/src/backend/storage/smgr/smgr.c
index dc466e5414..5224ca5259 100644
--- a/src/backend/storage/smgr/smgr.c
+++ b/src/backend/storage/smgr/smgr.c
@@ -50,6 +50,8 @@ typedef struct f_smgr
+/*
+ * smgrzeroextend() -- Add new zeroed out blocks to a file.
+ *
+ * Similar to smgrextend(), except the relation can be extended by
+ * multiple blocks at once, and that the added blocks will be
filled with
+ * zeroes.
+ */

Similar grammatical feedback as mdzeroextend.

From ad7cd10a6c340d7f7d0adf26d5e39224dfd8439d Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Wed, 26 Oct 2022 12:05:07 -0700
Subject: [PATCH v5 05/15] bufmgr: Add Pin/UnpinLocalBuffer()

diff --git a/src/backend/storage/buffer/bufmgr.c
b/src/backend/storage/buffer/bufmgr.c
index fa20fab5a2..6f50dbd212 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -4288,18 +4268,16 @@ ConditionalLockBuffer(Buffer buffer)
}

void
-BufferCheckOneLocalPin(Buffer buffer)
+BufferCheckWePinOnce(Buffer buffer)

This name is weird. Who is we?

diff --git a/src/backend/storage/buffer/localbuf.c
b/src/backend/storage/buffer/localbuf.c
index 5325ddb663..798c5b93a8 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
+bool
+PinLocalBuffer(BufferDesc *buf_hdr, bool adjust_usagecount)
+{
+ uint32 buf_state;
+ Buffer buffer = BufferDescriptorGetBuffer(buf_hdr);
+ int bufid = -(buffer + 1);

You do
int buffid = -buffer - 1;
in UnpinLocalBuffer()
They should be consistent.

int bufid = -(buffer + 1);

I think this version is better:

int buffid = -buffer - 1;

Since if buffer is INT_MAX, then the -(buffer + 1) version invokes
undefined behavior while the -buffer - 1 version doesn't.

From a0228218e2ac299aac754eeb5b2be7ddfc56918d Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Fri, 17 Feb 2023 18:26:34 -0800
Subject: [PATCH v5 07/15] bufmgr: Acquire and clean victim buffer separately

Previously we held buffer locks for two buffer mapping partitions at the same
time to change the identity of buffers. Particularly for extending relations
needing to hold the extension lock while acquiring a victim buffer is
painful. By separating out the victim buffer acquisition, future commits will
be able to change relation extensions to scale better.

diff --git a/src/backend/storage/buffer/bufmgr.c
b/src/backend/storage/buffer/bufmgr.c
index 3d0683593f..ea423ae484 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
@@ -1200,293 +1200,111 @@ BufferAlloc(SMgrRelation smgr, char
relpersistence, ForkNumber forkNum,

/*
* Buffer contents are currently invalid. Try to obtain the right to
* start I/O. If StartBufferIO returns false, then someone else managed
* to read it before we did, so there's nothing left for BufferAlloc() to
* do.
*/
- if (StartBufferIO(buf, true))
+ if (StartBufferIO(victim_buf_hdr, true))
*foundPtr = false;
else
*foundPtr = true;

I know it was already like this, but since you edited the line already,
can we just make this this now?

*foundPtr = !StartBufferIO(victim_buf_hdr, true);

@@ -1595,6 +1413,237 @@ retry:
StrategyFreeBuffer(buf);
}

+/*
+ * Helper routine for GetVictimBuffer()
+ *
+ * Needs to be called on a buffer with a valid tag, pinned, but without the
+ * buffer header spinlock held.
+ *
+ * Returns true if the buffer can be reused, in which case the buffer is only
+ * pinned by this backend and marked as invalid, false otherwise.
+ */
+static bool
+InvalidateVictimBuffer(BufferDesc *buf_hdr)
+{
+ /*
+ * Clear out the buffer's tag and flags and usagecount. This is not
+ * strictly required, as BM_TAG_VALID/BM_VALID needs to be checked before
+ * doing anything with the buffer. But currently it's beneficial as the
+ * pre-check for several linear scans of shared buffers just checks the
+ * tag.

I don't really understand the above comment -- mainly the last sentence.

+static Buffer
+GetVictimBuffer(BufferAccessStrategy strategy, IOContext io_context)
+{
+ BufferDesc *buf_hdr;
+ Buffer buf;
+ uint32 buf_state;
+ bool from_ring;
+
+ /*
+ * Ensure, while the spinlock's not yet held, that there's a free refcount
+ * entry.
+ */
+ ReservePrivateRefCountEntry();
+ ResourceOwnerEnlargeBuffers(CurrentResourceOwner);
+
+ /* we return here if a prospective victim buffer gets used concurrently */
+again:

Why use goto instead of a loop here (again is the goto label)?

From a7597b79dffaf96807f4a9beea0a39634530298d Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Mon, 24 Oct 2022 16:44:16 -0700
Subject: [PATCH v5 08/15] bufmgr: Support multiple in-progress IOs by using
resowner

Commit message should describe why we couldn't support multiple
in-progress IOs before, I think (e.g. we couldn't be sure that we
cleared IO_IN_PROGRESS if something happened).

@@ -4709,8 +4704,6 @@ TerminateBufferIO(BufferDesc *buf, bool
clear_dirty, uint32 set_flag_bits)
{
uint32 buf_state;

I noticed that the comment above TermianteBufferIO() says
* TerminateBufferIO: release a buffer we were doing I/O on
* (Assumptions)
* My process is executing IO for the buffer

Can we still say this is an assumption? What about when it is being
cleaned up after being called from AbortBufferIO()

diff --git a/src/backend/utils/resowner/resowner.c
b/src/backend/utils/resowner/resowner.c
index 19b6241e45..fccc59b39d 100644
--- a/src/backend/utils/resowner/resowner.c
+++ b/src/backend/utils/resowner/resowner.c
@@ -121,6 +121,7 @@ typedef struct ResourceOwnerData

/* We have built-in support for remembering: */
ResourceArray bufferarr; /* owned buffers */
+ ResourceArray bufferioarr; /* in-progress buffer IO */
ResourceArray catrefarr; /* catcache references */
ResourceArray catlistrefarr; /* catcache-list pins */
ResourceArray relrefarr; /* relcache references */
@@ -441,6 +442,7 @@ ResourceOwnerCreate(ResourceOwner parent, const char *name)

Maybe worth mentioning in-progress buffer IO in resowner README? I know
it doesn't claim to be exhaustive, so, up to you.

Also, I realize that existing code in this file has the extraneous
parantheses, but maybe it isn't worth staying consistent with that?
as in: &(owner->bufferioarr)

+ */
+void
+ResourceOwnerRememberBufferIO(ResourceOwner owner, Buffer buffer)
+{
+ ResourceArrayAdd(&(owner->bufferioarr), BufferGetDatum(buffer));
+}
+

From f26d1fa7e528d04436402aa8f94dc2442999dde3 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Wed, 1 Mar 2023 13:24:19 -0800
Subject: [PATCH v5 09/15] bufmgr: Move relation extension handling into
ExtendBufferedRel{By,To,}

diff --git a/src/backend/storage/buffer/bufmgr.c
b/src/backend/storage/buffer/bufmgr.c
index 3c95b87bca..4e07a5bc48 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c

+/*
+ * Extend relation by multiple blocks.
+ *
+ * Tries to extend the relation by extend_by blocks. Depending on the
+ * availability of resources the relation may end up being extended by a
+ * smaller number of pages (unless an error is thrown, always by at least one
+ * page). *extended_by is updated to the number of pages the relation has been
+ * extended to.
+ *
+ * buffers needs to be an array that is at least extend_by long. Upon
+ * completion, the first extend_by array elements will point to a pinned
+ * buffer.
+ *
+ * If EB_LOCK_FIRST is part of flags, the first returned buffer is
+ * locked. This is useful for callers that want a buffer that is guaranteed to
+ * be empty.

This should document what the returned BlockNumber is.
Also, instead of having extend_by and extended_by, how about just having
one which is set by the caller to the desired number to extend by and
then overwritten in this function to the value it successfully extended
by.

It would be nice if the function returned the number it extended by
instead of the BlockNumber.

+ */
+BlockNumber
+ExtendBufferedRelBy(ExtendBufferedWhat eb,
+ ForkNumber fork,
+ BufferAccessStrategy strategy,
+ uint32 flags,
+ uint32 extend_by,
+ Buffer *buffers,
+ uint32 *extended_by)
+{
+ Assert((eb.rel != NULL) ^ (eb.smgr != NULL));

Can we turn these into !=

Assert((eb.rel != NULL) != (eb.smgr != NULL));

since it is easier to understand.

(it is also in ExtendBufferedRelTo())

+ Assert(eb.smgr == NULL || eb.relpersistence != 0);
+ Assert(extend_by > 0);
+
+ if (eb.smgr == NULL)
+ {
+ eb.smgr = RelationGetSmgr(eb.rel);
+ eb.relpersistence = eb.rel->rd_rel->relpersistence;
+ }
+
+ return ExtendBufferedRelCommon(eb, fork, strategy, flags,
+ extend_by, InvalidBlockNumber,
+ buffers, extended_by);
+}

+ * Extend the relation so it is at least extend_to blocks large, read buffer

Use of "read buffer" here is confusing. We only read the block if, after
we try extending the relation, someone else already did so and we have
to read the block they extended in, right?

+ * (extend_to - 1).
+ *
+ * This is useful for callers that want to write a specific page, regardless
+ * of the current size of the relation (e.g. useful for visibilitymap and for
+ * crash recovery).
+ */
+Buffer
+ExtendBufferedRelTo(ExtendBufferedWhat eb,
+ ForkNumber fork,
+ BufferAccessStrategy strategy,
+ uint32 flags,
+ BlockNumber extend_to,
+ ReadBufferMode mode)
+{

+ while (current_size < extend_to)
+ {

Can declare buffers variable here.
+ Buffer buffers[64];

+ uint32 num_pages = lengthof(buffers);
+ BlockNumber first_block;
+
+ if ((uint64) current_size + num_pages > extend_to)
+ num_pages = extend_to - current_size;
+
+ first_block = ExtendBufferedRelCommon(eb, fork, strategy, flags,
+ num_pages, extend_to,
+ buffers, &extended_by);
+
+ current_size = first_block + extended_by;
+ Assert(current_size <= extend_to);
+ Assert(num_pages != 0 || current_size >= extend_to);
+
+ for (int i = 0; i < extended_by; i++)
+ {
+ if (first_block + i != extend_to - 1)

Is there a way we could avoid pinning these other buffers to begin with
(e.g. passing a parameter to ExtendBufferedRelCommon())

+ ReleaseBuffer(buffers[i]);
+ else
+ buffer = buffers[i];
+ }
+ }

+ /*
+ * It's possible that another backend concurrently extended the
+ * relation. In that case read the buffer.
+ *
+ * XXX: Should we control this via a flag?
+ */

I feel like there needs to be a more explicit comment about how you
could end up in this situation -- e.g. someone else extends the relation
and so smgrnblocks returns a value that is greater than extend_to, so
buffer stays InvalidBuffer

+ if (buffer == InvalidBuffer)
+ {
+ bool hit;
+
+ Assert(extended_by == 0);
+ buffer = ReadBuffer_common(eb.smgr, eb.relpersistence,
+ fork, extend_to - 1, mode, strategy,
+ &hit);
+ }
+
+ return buffer;
+}

Do we use compound literals? Here, this could be:

buffer = ReadBuffer_common(eb.smgr, eb.relpersistence,
fork, extend_to - 1, mode, strategy,
&(bool) {0});

To eliminate the extraneous hit variable.

/*
* ReadBuffer_common -- common logic for all ReadBuffer variants
@@ -801,35 +991,36 @@ ReadBuffer_common(SMgrRelation smgr, char
relpersistence, ForkNumber forkNum,
bool found;
IOContext io_context;
IOObject io_object;
- 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.

Think these are old function names in the comment

+static BlockNumber
+ExtendBufferedRelShared(ExtendBufferedWhat eb,
+ ForkNumber fork,
+ BufferAccessStrategy strategy,
+ uint32 flags,
+ uint32 extend_by,
+ BlockNumber extend_upto,
+ Buffer *buffers,
+ uint32 *extended_by)
+{
+ BlockNumber first_block;
+ IOContext io_context = IOContextForStrategy(strategy);
+
+ LimitAdditionalPins(&extend_by);
+
+ /*
+ * 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 < extend_by; i++)
+ {
+ Block buf_block;
+
+ buffers[i] = GetVictimBuffer(strategy, io_context);
+ buf_block = BufHdrGetBlock(GetBufferDescriptor(buffers[i] - 1));
+
+ /* new buffers are zero-filled */
+ MemSet((char *) buf_block, 0, BLCKSZ);
+ }
+
+ /*
+ * Lock relation against concurrent extensions, unless requested not to.
+ *
+ * We use the same extension lock for all forks. That's unnecessarily
+ * restrictive, but currently extensions for forks don't happen often
+ * enough to make it worth locking more granularly.
+ *
+ * Note that another backend might have extended the relation by the time
+ * we get the lock.
+ */
+ if (!(flags & EB_SKIP_EXTENSION_LOCK))
+ {
+ LockRelationForExtension(eb.rel, ExclusiveLock);
+ eb.smgr = RelationGetSmgr(eb.rel);
+ }
+
+ /*
+ * If requested, invalidate size cache, so that smgrnblocks asks the
+ * kernel.
+ */
+ if (flags & EB_CLEAR_SIZE_CACHE)
+ eb.smgr->smgr_cached_nblocks[fork] = InvalidBlockNumber;

I don't see this in master, is it new?

+ first_block = smgrnblocks(eb.smgr, fork);
+

The below needs a better comment explaining what it is handling. e.g. if
we end up extending by less than we planned, unpin all of the surplus
victim buffers.

+ if (extend_upto != InvalidBlockNumber)
+ {
+ uint32 old_num_pages = extend_by;

maybe call this something like original_extend_by

diff --git a/src/backend/storage/buffer/localbuf.c
b/src/backend/storage/buffer/localbuf.c
index 5b44b0be8b..0528fddf99 100644
--- a/src/backend/storage/buffer/localbuf.c
+++ b/src/backend/storage/buffer/localbuf.c
+BlockNumber
+ExtendBufferedRelLocal(ExtendBufferedWhat eb,
+ ForkNumber fork,
+ uint32 flags,
+ uint32 extend_by,
+ BlockNumber extend_upto,
+ Buffer *buffers,
+ uint32 *extended_by)
+{

+ victim_buf_id = -(buffers[i] + 1);

same comment here as before.

+ * Flags influencing the behaviour of ExtendBufferedRel*
+ */
+typedef enum ExtendBufferedFlags
+{
+ /*
+ * Don't acquire extension lock. This is safe only if the relation isn't
+ * shared, an access exclusive lock is held or if this is the startup
+ * process.
+ */
+ EB_SKIP_EXTENSION_LOCK = (1 << 0),
+
+ /* Is this extension part of recovery? */
+ EB_PERFORMING_RECOVERY = (1 << 1),
+
+ /*
+ * Should the fork be created if it does not currently exist? This likely
+ * only ever makes sense for relation forks.
+ */
+ EB_CREATE_FORK_IF_NEEDED = (1 << 2),
+
+ /* Should the first (possibly only) return buffer be returned locked? */
+ EB_LOCK_FIRST = (1 << 3),
+
+ /* Should the smgr size cache be cleared? */
+ EB_CLEAR_SIZE_CACHE = (1 << 4),
+
+ /* internal flags follow */

I don't understand what this comment means ("internal flags follow")

+ EB_LOCK_TARGET = (1 << 5),
+} ExtendBufferedFlags;

+typedef struct ExtendBufferedWhat

Maybe this should be called like BufferedExtendTarget or something?

+{
+ Relation rel;
+ struct SMgrRelationData *smgr;
+ char relpersistence;
+} ExtendBufferedWhat;

From e4438c0eb87035e4cefd1de89458a8d88c90c0e3 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Sun, 23 Oct 2022 14:44:43 -0700
Subject: [PATCH v5 11/15] heapam: Add num_pages to RelationGetBufferForTuple()

This will be useful to compute the number of pages to extend a relation by.

diff --git a/src/backend/access/heap/heapam.c b/src/backend/access/heap/heapam.c
index cf4b917eb4..500904897d 100644
--- a/src/backend/access/heap/heapam.c
+++ b/src/backend/access/heap/heapam.c
@@ -2050,6 +2053,33 @@ heap_prepare_insert(Relation relation,
HeapTuple tup, TransactionId xid,
return tup;
}

+/*
+ * Helper for heap_multi_insert() that computes the number of full pages s

no space after page before s

+ */
+static int
+heap_multi_insert_pages(HeapTuple *heaptuples, int done, int ntuples,
Size saveFreeSpace)
+{
+ size_t page_avail;
+ int npages = 0;
+
+ page_avail = BLCKSZ - SizeOfPageHeaderData - saveFreeSpace;
+ npages++;

can this not just be this:

size_t page_avail = BLCKSZ - SizeOfPageHeaderData - saveFreeSpace;
int npages = 1;

From 5d2be27caf8f4ee8f26841b2aa1674c90bd51754 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Wed, 26 Oct 2022 14:14:11 -0700
Subject: [PATCH v5 12/15] hio: Use ExtendBufferedRelBy()

---
src/backend/access/heap/hio.c | 285 +++++++++++++++++-----------------
1 file changed, 146 insertions(+), 139 deletions(-)

diff --git a/src/backend/access/heap/hio.c b/src/backend/access/heap/hio.c
index 65886839e7..48cfcff975 100644
--- a/src/backend/access/heap/hio.c
+++ b/src/backend/access/heap/hio.c
@@ -354,6 +270,9 @@ RelationGetBufferForTuple(Relation relation, Size len,

so in RelationGetBufferForTuple() up above where your changes start,
there is this code

/*
* We first try to put the tuple on the same page we last inserted a tuple
* on, as cached in the BulkInsertState or relcache entry. If that
* doesn't work, we ask the Free Space Map to locate a suitable page.
* Since the FSM's info might be out of date, we have to be prepared to
* loop around and retry multiple times. (To insure this isn't an infinite
* loop, we must update the FSM with the correct amount of free space on
* each page that proves not to be suitable.) If the FSM has no record of
* a page with enough free space, we give up and extend the relation.
*
* When use_fsm is false, we either put the tuple onto the existing target
* page or extend the relation.
*/
if (bistate && bistate->current_buf != InvalidBuffer)
{
targetBlock = BufferGetBlockNumber(bistate->current_buf);
}
else
targetBlock = RelationGetTargetBlock(relation);

if (targetBlock == InvalidBlockNumber && use_fsm)
{
/*
* We have no cached target page, so ask the FSM for an initial
* target.
*/
targetBlock = GetPageWithFreeSpace(relation, targetFreeSpace);
}

And, I was thinking how, ReadBufferBI() only has one caller now
(RelationGetBufferForTuple()) and, this caller basically already has
checked for the case in the inside of ReadBufferBI() (the code I pasted
above)

/* If we have the desired block already pinned, re-pin and return it */
if (bistate->current_buf != InvalidBuffer)
{
if (BufferGetBlockNumber(bistate->current_buf) == targetBlock)
{
/*
* Currently the LOCK variants are only used for extending
* relation, which should never reach this branch.
*/
Assert(mode != RBM_ZERO_AND_LOCK &&
mode != RBM_ZERO_AND_CLEANUP_LOCK);

IncrBufferRefCount(bistate->current_buf);
return bistate->current_buf;
}
/* ... else drop the old buffer */

So, I was thinking maybe there is some way to inline the logic for
ReadBufferBI(), because I think it would feel more streamlined to me.

@@ -558,18 +477,46 @@ loop:
ReleaseBuffer(buffer);
}

Oh, and I forget which commit introduced BulkInsertState->next_free and
last_free, but I remember thinking that it didn't seem to fit with the
other parts of that commit.

- /* Without FSM, always fall out of the loop and extend */
- if (!use_fsm)
- break;
+ if (bistate
+ && bistate->next_free != InvalidBlockNumber
+ && bistate->next_free <= bistate->last_free)
+ {
+ /*
+ * We bulk extended the relation before, and there are still some
+ * unused pages from that extension, so we don't need to look in
+ * the FSM for a new page. But do record the free space from the
+ * last page, somebody might insert narrower tuples later.
+ */

Why couldn't we have found out that we bulk-extended before and get the
block from there up above the while loop?

+ if (use_fsm)
+ RecordPageWithFreeSpace(relation, targetBlock, pageFreeSpace);

- /*
- * Update FSM as to condition of this page, and ask for another page
- * to try.
- */
- targetBlock = RecordAndGetPageWithFreeSpace(relation,
- targetBlock,
- pageFreeSpace,
- targetFreeSpace);
+ Assert(bistate->last_free != InvalidBlockNumber &&

You don't need the below half of the assert.

+ bistate->next_free <= bistate->last_free);
+ targetBlock = bistate->next_free;
+ if (bistate->next_free >= bistate->last_free)

they can only be equal at this point

+ {
+ bistate->next_free = InvalidBlockNumber;
+ bistate->last_free = InvalidBlockNumber;
+ }
+ else
+ bistate->next_free++;
+ }
+ else if (!use_fsm)
+ {
+ /* Without FSM, always fall out of the loop and extend */
+ break;
+ }

It would be nice to have a comment explaining why this is in its own
else if instead of breaking earlier (i.e. !use_fsm is still a valid case
in the if branch above it)

+ else
+ {
+ /*
+ * Update FSM as to condition of this page, and ask for another
+ * page to try.
+ */
+ targetBlock = RecordAndGetPageWithFreeSpace(relation,
+ targetBlock,
+ pageFreeSpace,
+ targetFreeSpace);
+ }

we can get rid of needLock and waitcount variables like this

+#define MAX_BUFFERS 64
+ Buffer victim_buffers[MAX_BUFFERS];
+ BlockNumber firstBlock = InvalidBlockNumber;
+ BlockNumber firstBlockFSM = InvalidBlockNumber;
+ BlockNumber curBlock;
+ uint32 extend_by_pages;
+ uint32 no_fsm_pages;
+ uint32 waitcount;
+
+ extend_by_pages = num_pages;
+
+ /*
+ * Multiply the number of pages to extend by the number of waiters. Do
+ * this even if we're not using the FSM, as it does relieve
+ * contention. Pages will be found via bistate->next_free.
+ */
+ if (needLock)
+ waitcount = RelationExtensionLockWaiterCount(relation);
+ else
+ waitcount = 0;
+ extend_by_pages += extend_by_pages * waitcount;

if (!RELATION_IS_LOCAL(relation))
extend_by_pages += extend_by_pages *
RelationExtensionLockWaiterCount(relation);

+
+ /*
+ * can't extend by more than MAX_BUFFERS, we need to pin them all
+ * concurrently. FIXME: Need an NBuffers / MaxBackends type limit
+ * here.
+ */
+ extend_by_pages = Min(extend_by_pages, MAX_BUFFERS);
+
+ /*
+ * How many of the extended pages not to enter into the FSM.
+ *
+ * Only enter pages that we don't need ourselves into the FSM.
+ * Otherwise every other backend will immediately try to use the pages
+ * this backend neds itself, causing unnecessary contention.
+ *
+ * Bulk extended pages are remembered in bistate->next_free_buffer. So
+ * without a bistate we can't directly make use of them.
+ *
+ * Never enter the page returned into the FSM, we'll immediately use
+ * it.
+ */
+ if (num_pages > 1 && bistate == NULL)
+ no_fsm_pages = 1;
+ else
+ no_fsm_pages = num_pages;

this is more clearly this:
no_fsm_pages = bistate == NULL ? 1 : num_pages;

- /*
- * Release the file-extension lock; it's now OK for someone else to extend
- * the relation some more.
- */
- if (needLock)
- UnlockRelationForExtension(relation, ExclusiveLock);
+ if (bistate)
+ {
+ if (extend_by_pages > 1)
+ {
+ bistate->next_free = firstBlock + 1;
+ bistate->last_free = firstBlock + extend_by_pages - 1;
+ }
+ else
+ {
+ bistate->next_free = InvalidBlockNumber;
+ bistate->last_free = InvalidBlockNumber;
+ }
+ }
+
+ buffer = victim_buffers[0];

If we move buffer = up, we can have only one if (bistate)

+ if (bistate)
+ {
+ IncrBufferRefCount(buffer);
+ bistate->current_buf = buffer;
+ }
+ }

like this:

buffer = victim_buffers[0];

if (bistate)
{
if (extend_by_pages > 1)
{
bistate->next_free = firstBlock + 1;
bistate->last_free = firstBlock + extend_by_pages - 1;
}
else
{
bistate->next_free = InvalidBlockNumber;
bistate->last_free = InvalidBlockNumber;
}

IncrBufferRefCount(buffer);
bistate->current_buf = buffer;
}

From 6711e45bed59ee07ec277b9462f4745603a3d4a4 Mon Sep 17 00:00:00 2001
From: Andres Freund <andres(at)anarazel(dot)de>
Date: Sun, 23 Oct 2022 14:41:46 -0700
Subject: [PATCH v5 15/15] bufmgr: debug: Add PrintBuffer[Desc]

Useful for development. Perhaps we should polish these and keep them?
diff --git a/src/backend/storage/buffer/bufmgr.c
b/src/backend/storage/buffer/bufmgr.c
index 4e07a5bc48..0d382cd787 100644
--- a/src/backend/storage/buffer/bufmgr.c
+++ b/src/backend/storage/buffer/bufmgr.c
+
+ fprintf(stderr, "%d: [%u] msg: %s, rel: %s, block %u: refcount:
%u / %u, usagecount: %u, flags:%s%s%s%s%s%s%s%s%s%s\n",
+ MyProcPid,
+ buffer,
+ msg,
+ path,
+ blockno,
+ BUF_STATE_GET_REFCOUNT(buf_state),
+ GetPrivateRefCount(buffer),
+ BUF_STATE_GET_USAGECOUNT(buf_state),
+ buf_state & BM_LOCKED ? " BM_LOCKED" : "",
+ buf_state & BM_DIRTY ? " BM_DIRTY" : "",
+ buf_state & BM_VALID ? " BM_VALID" : "",
+ buf_state & BM_TAG_VALID ? " BM_TAG_VALID" : "",
+ buf_state & BM_IO_IN_PROGRESS ? " BM_IO_IN_PROGRESS" : "",
+ buf_state & BM_IO_ERROR ? " BM_IO_ERROR" : "",
+ buf_state & BM_JUST_DIRTIED ? " BM_JUST_DIRTIED" : "",
+ buf_state & BM_PIN_COUNT_WAITER ? " BM_PIN_COUNT_WAITER" : "",
+ buf_state & BM_CHECKPOINT_NEEDED ? " BM_CHECKPOINT_NEEDED" : "",
+ buf_state & BM_PERMANENT ? " BM_PERMANENT" : ""
+ );
+}

How about this

#define FLAG_DESC(flag) (buf_state & (flag) ? " " #flag : "")
FLAG_DESC(BM_LOCKED),
FLAG_DESC(BM_DIRTY),
FLAG_DESC(BM_VALID),
FLAG_DESC(BM_TAG_VALID),
FLAG_DESC(BM_IO_IN_PROGRESS),
FLAG_DESC(BM_IO_ERROR),
FLAG_DESC(BM_JUST_DIRTIED),
FLAG_DESC(BM_PIN_COUNT_WAITER),
FLAG_DESC(BM_CHECKPOINT_NEEDED),
FLAG_DESC(BM_PERMANENT)
#undef FLAG_DESC

+
+void
+PrintBuffer(Buffer buffer, const char *msg)
+{
+ BufferDesc *buf_hdr = GetBufferDescriptor(buffer - 1);

no need for this variable

+
+ PrintBufferDesc(buf_hdr, msg);
+}

- Melanie

In response to

Responses

Browse pgsql-hackers by date

  From Date Subject
Next Message Dave Cramer 2023-03-26 21:54:05 Re: Request for comment on setting binary format output per session
Previous Message Andres Freund 2023-03-26 21:40:20 Re: Time to move pg_test_timing to measure in nanoseconds